Merge branch 'gerrit/goldfish' into 'master'

Change-Id: I65b6f845e9958fed620fffb78837ab07f1f03559
diff --git a/asterixdb/LICENSE b/asterixdb/LICENSE
index 4c63d01..72413f8 100644
--- a/asterixdb/LICENSE
+++ b/asterixdb/LICENSE
@@ -517,6 +517,202 @@
    limitations under the License.
 ---
 
+   Portions of the AsterixDB OM
+       located at:
+         asterix-app/src/main/resources/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/JtsModule.java,
+         asterix-app/src/main/resources/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/MultiLineStringParser.java,
+         asterix-app/src/main/resources/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/PointParser.java,
+         asterix-app/src/main/resources/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/MultiPointParser.java,
+         asterix-app/src/main/resources/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/GeometryParser.java,
+         asterix-app/src/main/resources/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/MultiPolygonParser.java,
+         asterix-app/src/main/resources/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/LineStringParser.java,
+         asterix-app/src/main/resources/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/BaseParser.java,
+         asterix-app/src/main/resources/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/GeometryCollectionParser.java,
+         asterix-app/src/main/resources/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/GenericGeometryParser.java,
+         asterix-app/src/main/resources/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/PolygonParser.java,
+         asterix-app/src/main/resources/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/GeoFunctionUtils.java,
+         asterix-app/src/main/resources/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/GeoJsonConstants.java,
+         asterix-app/src/main/resources/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/serde/GeometryDeserializer.java,
+       and
+         asterix-app/src/main/resources/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/serde/GeometrySerializer.java
+
+   are available under the following license:
+---
+   Classes are modified to support the org.locationtech.jts package instead of com.vividsolutions.jts
+   Copyright BeDataDriven
+
+   Apache License
+   Version 2.0, January 2004
+   http://www.apache.org/licenses/
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+   1. Definitions.
+   "License" shall mean the terms and conditions for use, reproduction,
+   and distribution as defined by Sections 1 through 9 of this document.
+   "Licensor" shall mean the copyright owner or entity authorized by
+   the copyright owner that is granting the License.
+   "Legal Entity" shall mean the union of the acting entity and all
+   other entities that control, are controlled by, or are under common
+   control with that entity. For the purposes of this definition,
+   "control" means (i) the power, direct or indirect, to cause the
+   direction or management of such entity, whether by contract or
+   otherwise, or (ii) ownership of fifty percent (50%) or more of the
+   outstanding shares, or (iii) beneficial ownership of such entity.
+   "You" (or "Your") shall mean an individual or Legal Entity
+   exercising permissions granted by this License.
+   "Source" form shall mean the preferred form for making modifications,
+   including but not limited to software source code, documentation
+   source, and configuration files.
+   "Object" form shall mean any form resulting from mechanical
+   transformation or translation of a Source form, including but
+   not limited to compiled object code, generated documentation,
+   and conversions to other media types.
+   "Work" shall mean the work of authorship, whether in Source or
+   Object form, made available under the License, as indicated by a
+   copyright notice that is included in or attached to the work
+   (an example is provided in the Appendix below).
+   "Derivative Works" shall mean any work, whether in Source or Object
+   form, that is based on (or derived from) the Work and for which the
+   editorial revisions, annotations, elaborations, or other modifications
+   represent, as a whole, an original work of authorship. For the purposes
+   of this License, Derivative Works shall not include works that remain
+   separable from, or merely link (or bind by name) to the interfaces of,
+   the Work and Derivative Works thereof.
+   "Contribution" shall mean any work of authorship, including
+   the original version of the Work and any modifications or additions
+   to that Work or Derivative Works thereof, that is intentionally
+   submitted to Licensor for inclusion in the Work by the copyright owner
+   or by an individual or Legal Entity authorized to submit on behalf of
+   the copyright owner. For the purposes of this definition, "submitted"
+   means any form of electronic, verbal, or written communication sent
+   to the Licensor or its representatives, including but not limited to
+   communication on electronic mailing lists, source code control systems,
+   and issue tracking systems that are managed by, or on behalf of, the
+   Licensor for the purpose of discussing and improving the Work, but
+   excluding communication that is conspicuously marked or otherwise
+   designated in writing by the copyright owner as "Not a Contribution."
+   "Contributor" shall mean Licensor and any individual or Legal Entity
+   on behalf of whom a Contribution has been received by Licensor and
+   subsequently incorporated within the Work.
+   2. Grant of Copyright License. Subject to the terms and conditions of
+   this License, each Contributor hereby grants to You a perpetual,
+   worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+   copyright license to reproduce, prepare Derivative Works of,
+   publicly display, publicly perform, sublicense, and distribute the
+   Work and such Derivative Works in Source or Object form.
+   3. Grant of Patent License. Subject to the terms and conditions of
+   this License, each Contributor hereby grants to You a perpetual,
+   worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+   (except as stated in this section) patent license to make, have made,
+   use, offer to sell, sell, import, and otherwise transfer the Work,
+   where such license applies only to those patent claims licensable
+   by such Contributor that are necessarily infringed by their
+   Contribution(s) alone or by combination of their Contribution(s)
+   with the Work to which such Contribution(s) was submitted. If You
+   institute patent litigation against any entity (including a
+   cross-claim or counterclaim in a lawsuit) alleging that the Work
+   or a Contribution incorporated within the Work constitutes direct
+   or contributory patent infringement, then any patent licenses
+   granted to You under this License for that Work shall terminate
+   as of the date such litigation is filed.
+   4. Redistribution. You may reproduce and distribute copies of the
+   Work or Derivative Works thereof in any medium, with or without
+   modifications, and in Source or Object form, provided that You
+   meet the following conditions:
+   (a) You must give any other recipients of the Work or
+   Derivative Works a copy of this License; and
+   (b) You must cause any modified files to carry prominent notices
+   stating that You changed the files; and
+   (c) You must retain, in the Source form of any Derivative Works
+   that You distribute, all copyright, patent, trademark, and
+   attribution notices from the Source form of the Work,
+   excluding those notices that do not pertain to any part of
+   the Derivative Works; and
+   (d) If the Work includes a "NOTICE" text file as part of its
+   distribution, then any Derivative Works that You distribute must
+   include a readable copy of the attribution notices contained
+   within such NOTICE file, excluding those notices that do not
+   pertain to any part of the Derivative Works, in at least one
+   of the following places: within a NOTICE text file distributed
+   as part of the Derivative Works; within the Source form or
+   documentation, if provided along with the Derivative Works; or,
+   within a display generated by the Derivative Works, if and
+   wherever such third-party notices normally appear. The contents
+   of the NOTICE file are for informational purposes only and
+   do not modify the License. You may add Your own attribution
+   notices within Derivative Works that You distribute, alongside
+   or as an addendum to the NOTICE text from the Work, provided
+   that such additional attribution notices cannot be construed
+   as modifying the License.
+   You may add Your own copyright statement to Your modifications and
+   may provide additional or different license terms and conditions
+   for use, reproduction, or distribution of Your modifications, or
+   for any such Derivative Works as a whole, provided Your use,
+   reproduction, and distribution of the Work otherwise complies with
+   the conditions stated in this License.
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+   any Contribution intentionally submitted for inclusion in the Work
+   by You to the Licensor shall be under the terms and conditions of
+   this License, without any additional terms or conditions.
+   Notwithstanding the above, nothing herein shall supersede or modify
+   the terms of any separate license agreement you may have executed
+   with Licensor regarding such Contributions.
+   6. Trademarks. This License does not grant permission to use the trade
+   names, trademarks, service marks, or product names of the Licensor,
+   except as required for reasonable and customary use in describing the
+   origin of the Work and reproducing the content of the NOTICE file.
+   7. Disclaimer of Warranty. Unless required by applicable law or
+   agreed to in writing, Licensor provides the Work (and each
+   Contributor provides its Contributions) on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+   implied, including, without limitation, any warranties or conditions
+   of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+   PARTICULAR PURPOSE. You are solely responsible for determining the
+   appropriateness of using or redistributing the Work and assume any
+   risks associated with Your exercise of permissions under this License.
+   8. Limitation of Liability. In no event and under no legal theory,
+   whether in tort (including negligence), contract, or otherwise,
+   unless required by applicable law (such as deliberate and grossly
+   negligent acts) or agreed to in writing, shall any Contributor be
+   liable to You for damages, including any direct, indirect, special,
+   incidental, or consequential damages of any character arising as a
+   result of this License or out of the use or inability to use the
+   Work (including but not limited to damages for loss of goodwill,
+   work stoppage, computer failure or malfunction, or any and all
+   other commercial damages or losses), even if such Contributor
+   has been advised of the possibility of such damages.
+   9. Accepting Warranty or Additional Liability. While redistributing
+   the Work or Derivative Works thereof, You may choose to offer,
+   and charge a fee for, acceptance of support, warranty, indemnity,
+   or other liability obligations and/or rights consistent with this
+   License. However, in accepting such obligations, You may act only
+   on Your own behalf and on Your sole responsibility, not on behalf
+   of any other Contributor, and only if You agree to indemnify,
+   defend, and hold each Contributor harmless for any liability
+   incurred by, or claims asserted against, such Contributor by reason
+   of your accepting any such warranty or additional liability.
+   END OF TERMS AND CONDITIONS
+   APPENDIX: How to apply the Apache License to your work.
+   To apply the Apache License to your work, attach the following
+   boilerplate notice, with the fields enclosed by brackets "[]"
+   replaced with your own identifying information. (Don't include
+   the brackets!)  The text should be enclosed in the appropriate
+   comment syntax for the file format. We also recommend that a
+   file or class name and description of purpose be included on the
+   same "printed page" as the copyright notice for easier
+   identification within third-party archives.
+   Copyright [yyyy] [name of copyright owner]
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+   http://www.apache.org/licenses/LICENSE-2.0
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+   For more details on the original project and licensing, please visit
+   https://github.com/bedatadriven/jackson-datatype-jts.
+---
    Portions of the AsterixDB API examples
        located at:
          asterix-examples/src/main/resources/admaql101-demo/bottle.py,
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/AssignBatchPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/AssignBatchPOperator.java
index 0ec5ee7..6ec9f7b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/AssignBatchPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/AssignBatchPOperator.java
@@ -74,13 +74,13 @@
                         fi.toString());
             }
             fnDescs[i] = ExternalFunctionDescriptorProvider.resolveExternalFunction(callExpr, inputTypeEnv, context);
-            fnArgColumns[i] = getColumns(callExpr.getArguments(), opSchema, op.getSourceLocation());
+            fnArgColumns[i] = getColumns(callExpr.getArguments(), inputSchemas[0], op.getSourceLocation());
         }
 
         return new ExternalAssignBatchRuntimeFactory(outColumns, fnDescs, fnArgColumns, projectionList);
     }
 
-    private int[] getColumns(List<Mutable<ILogicalExpression>> exprList, IOperatorSchema opSchema,
+    private int[] getColumns(List<Mutable<ILogicalExpression>> exprList, IOperatorSchema inputSchema,
             SourceLocation sourceLoc) throws CompilationException {
         int n = exprList.size();
         int[] columns = new int[n];
@@ -92,7 +92,7 @@
             }
             VariableReferenceExpression argVarRef = (VariableReferenceExpression) expr;
             LogicalVariable argVar = argVarRef.getVariableReference();
-            int argColumn = opSchema.findVariable(argVar);
+            int argColumn = inputSchema.findVariable(argVar);
             if (argColumn < 0) {
                 throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, String.valueOf(argVar));
             }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java
index 4469dbd..10574b1 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java
@@ -90,16 +90,17 @@
                 CompilerProperties.COMPILER_SORT_SAMPLES_KEY, CompilerProperties.COMPILER_EXTERNALSCANMEMORY_KEY,
                 CompilerProperties.COMPILER_INDEXONLY_KEY, CompilerProperties.COMPILER_INTERNAL_SANITYCHECK_KEY,
                 CompilerProperties.COMPILER_EXTERNAL_FIELD_PUSHDOWN_KEY, CompilerProperties.COMPILER_SUBPLAN_MERGE_KEY,
-                CompilerProperties.COMPILER_SUBPLAN_NESTEDPUSHDOWN_KEY, CompilerProperties.COMPILER_ARRAYINDEX_KEY,
-                CompilerProperties.COMPILER_CBO_KEY, CompilerProperties.COMPILER_CBO_TEST_KEY,
-                CompilerProperties.COMPILER_FORCE_JOIN_ORDER_KEY, CompilerProperties.COMPILER_QUERY_PLAN_SHAPE_KEY,
-                CompilerProperties.COMPILER_MIN_MEMORY_ALLOCATION_KEY, CompilerProperties.COMPILER_COLUMN_FILTER_KEY,
-                CompilerProperties.COMPILER_BATCH_LOOKUP_KEY, FunctionUtil.IMPORT_PRIVATE_FUNCTIONS,
-                FuzzyUtils.SIM_FUNCTION_PROP_NAME, FuzzyUtils.SIM_THRESHOLD_PROP_NAME,
-                StartFeedStatement.WAIT_FOR_COMPLETION, FeedActivityDetails.FEED_POLICY_NAME,
-                FeedActivityDetails.COLLECT_LOCATIONS, SqlppQueryRewriter.INLINE_WITH_OPTION,
-                SqlppExpressionToPlanTranslator.REWRITE_IN_AS_OR_OPTION, "hash_merge", "output-record-type",
-                DisjunctivePredicateToJoinRule.REWRITE_OR_AS_JOIN_OPTION,
+                CompilerProperties.COMPILER_SUBPLAN_NESTEDPUSHDOWN_KEY, CompilerProperties.COMPILER_ORDERED_FIELDS_KEY,
+                CompilerProperties.COMPILER_ARRAYINDEX_KEY, CompilerProperties.COMPILER_CBO_KEY,
+                CompilerProperties.COMPILER_CBO_TEST_KEY, CompilerProperties.COMPILER_FORCE_JOIN_ORDER_KEY,
+                CompilerProperties.COMPILER_QUERY_PLAN_SHAPE_KEY, CompilerProperties.COMPILER_MIN_MEMORY_ALLOCATION_KEY,
+                CompilerProperties.COMPILER_COLUMN_FILTER_KEY, CompilerProperties.COMPILER_BATCH_LOOKUP_KEY,
+                FunctionUtil.IMPORT_PRIVATE_FUNCTIONS,
+                CompilerProperties.COMPILER_MAX_VARIABLE_OCCURRENCES_INLINING_KEY, FuzzyUtils.SIM_FUNCTION_PROP_NAME,
+                FuzzyUtils.SIM_THRESHOLD_PROP_NAME, StartFeedStatement.WAIT_FOR_COMPLETION,
+                FeedActivityDetails.FEED_POLICY_NAME, FeedActivityDetails.COLLECT_LOCATIONS,
+                SqlppQueryRewriter.INLINE_WITH_OPTION, SqlppExpressionToPlanTranslator.REWRITE_IN_AS_OR_OPTION,
+                "hash_merge", "output-record-type", DisjunctivePredicateToJoinRule.REWRITE_OR_AS_JOIN_OPTION,
                 SetAsterixPhysicalOperatorsRule.REWRITE_ATTEMPT_BATCH_ASSIGN,
                 EquivalenceClassUtils.REWRITE_INTERNAL_QUERYUID_PK, SqlppQueryRewriter.SQL_COMPAT_OPTION,
                 JoinEnum.CBO_FULL_ENUM_LEVEL_KEY, JoinEnum.CBO_CP_ENUM_KEY));
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index 5209b3e..d786e9d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -119,6 +119,7 @@
 import org.apache.hyracks.algebricks.rewriter.rules.ConsolidateSelectsRule;
 import org.apache.hyracks.algebricks.rewriter.rules.CopyLimitDownRule;
 import org.apache.hyracks.algebricks.rewriter.rules.EliminateGroupByEmptyKeyRule;
+import org.apache.hyracks.algebricks.rewriter.rules.EmbedProjectRule;
 import org.apache.hyracks.algebricks.rewriter.rules.EnforceOrderByAfterSubplan;
 import org.apache.hyracks.algebricks.rewriter.rules.EnforceStructuralPropertiesRule;
 import org.apache.hyracks.algebricks.rewriter.rules.ExtractCommonExpressionsRule;
@@ -466,6 +467,8 @@
         prepareForJobGenRewrites.add(new PopulateResultMetadataRule());
         prepareForJobGenRewrites.add(new AnnotateOperatorCostCardinalityRule());
         prepareForJobGenRewrites.add(new EnsureColumnarSupportedTypesRule());
+        // EmbedProjectRule should run last. The assumption is that this rules set will run once
+        prepareForJobGenRewrites.add(new EmbedProjectRule());
         return prepareForJobGenRewrites;
     }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
index dd8f630..df8d4cd 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -860,7 +860,7 @@
 
                 AbstractFunctionCallExpression theFieldAccessFunc;
                 LogicalVariable fieldVar = context.newVar();
-                if (fieldType == null) {
+                if (fieldType == null && !ATypeTag.ANY.equals(skType.getTypeTag())) {
                     // Open field. must prevent inlining to maintain the cast before the primaryOp and
                     // make handling of records with incorrect value type for this field easier and cleaner
                     context.addNotToBeInlinedVar(fieldVar);
@@ -906,6 +906,9 @@
         FunctionIdentifier skFun = null;
         IAObject fmtArg = null;
         Pair<FunctionIdentifier, IAObject> castExpr;
+        if (ATypeTag.ANY.equals(skType.getTypeTag())) {
+            return new IndexFieldId(skSrc, skName, skType.getTypeTag(), skFun, fmtArg);
+        }
         if (fieldType == null) {
             // open field
             castExpr = getCastExpression(index, skType, srcLoc);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
index b5041f0..24ef2d7 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
@@ -485,7 +485,9 @@
 
                     for (int j = 0; j < matchedTypes.size(); j++) {
                         for (int k = j + 1; k < matchedTypes.size(); k++) {
-                            typeMatch &= isMatched(elementTypes.get(j), elementTypes.get(k), jaccardSimilarity);
+                            if (!ATypeTag.ANY.equals(elementTypes.get(k).getTypeTag())) {
+                                typeMatch &= isMatched(elementTypes.get(j), elementTypes.get(k), jaccardSimilarity);
+                            }
                         }
                     }
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
index 1ff90c6..9b30dfb 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
@@ -1133,20 +1133,40 @@
         }
     }
 
-    private static AbstractUnnestMapOperator createFinalNonIndexOnlySearchPlan(Dataset dataset,
-            ILogicalOperator inputOp, IOptimizationContext context, boolean sortPrimaryKeys, boolean retainInput,
-            boolean retainMissing, boolean requiresBroadcast, boolean requiresDistinct,
-            List<LogicalVariable> primaryKeyVars, List<LogicalVariable> primaryIndexUnnestVars,
-            List<LogicalVariable> auxDistinctVars, List<Object> primaryIndexOutputTypes,
-            IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
-        SourceLocation sourceLoc = inputOp.getSourceLocation();
+    private static AbstractUnnestMapOperator createFinalNonIndexOnlySearchPlan(
+            List<Mutable<ILogicalOperator>> afterTopOpRefs, Dataset dataset, ILogicalOperator inputOp,
+            IOptimizationContext context, boolean sortPrimaryKeys, boolean retainInput, boolean retainMissing,
+            boolean requiresBroadcast, boolean requiresDistinct, List<LogicalVariable> primaryKeyVars,
+            List<LogicalVariable> primaryIndexUnnestVars, List<LogicalVariable> auxDistinctVars,
+            List<Object> primaryIndexOutputTypes, IAlgebricksConstantValue leftOuterMissingValue,
+            ARecordType recordType, ARecordType metaRecordType, OptimizableOperatorSubTree subTree,
+            Index secondaryIndex, Mutable<ILogicalOperator> topOpRef,
+            List<Mutable<ILogicalOperator>> assignsBeforeTopOpRef, Mutable<ILogicalExpression> conditionRef,
+            LogicalVariable newMissingPlaceHolderForLOJ) throws AlgebricksException {
 
+        SourceLocation sourceLoc = inputOp.getSourceLocation();
         // Sanity check: requiresDistinct and sortPrimaryKeys are mutually exclusive.
         if (requiresDistinct && sortPrimaryKeys) {
             throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
                     "Non-index search plan " + "cannot include a DISTINCT and an ORDER.");
         }
+        ILogicalOperator op = null;
+        IndexType idxType = secondaryIndex.getIndexType();
 
+        // for now the additional selection is only added just for the value index.
+        if (Index.IndexCategory.of(idxType) == Index.IndexCategory.VALUE) {
+            Index.ValueIndexDetails secondaryIndexDetails = (Index.ValueIndexDetails) secondaryIndex.getIndexDetails();
+            List<List<String>> chosenIndexFieldNames = secondaryIndexDetails.getKeyFieldNames();
+            List<IAType> secodaryKeysType = secondaryIndexDetails.getKeyFieldTypes();
+            if (idxType == IndexType.BTREE && secodaryKeysType.contains(BuiltinType.ANY)) {
+                op = additionalSelectForHeterogeneousIndex(chosenIndexFieldNames, afterTopOpRefs, dataset, inputOp,
+                        context, retainMissing, leftOuterMissingValue, recordType, metaRecordType, subTree,
+                        secondaryIndex, topOpRef, assignsBeforeTopOpRef, conditionRef, newMissingPlaceHolderForLOJ);
+            }
+        }
+        if (op == null) {
+            op = inputOp;
+        }
         // If we have an array index, then we must only give unique keys to our primary-index scan.
         DistinctOperator distinct = null;
         if (requiresDistinct) {
@@ -1165,7 +1185,7 @@
             }
             distinct = new DistinctOperator(distinctExprs);
             distinct.setSourceLocation(sourceLoc);
-            distinct.getInputs().add(new MutableObject<>(inputOp));
+            distinct.getInputs().add(new MutableObject<>(op));
             distinct.setExecutionMode(ExecutionMode.LOCAL);
             context.computeAndSetTypeEnvironmentForOperator(distinct);
         }
@@ -1182,7 +1202,7 @@
                 order.getOrderExpressions().add(new Pair<>(OrderOperator.ASC_ORDER, vRef));
             }
             // The secondary-index search feeds into the sort.
-            order.getInputs().add(new MutableObject<>(inputOp));
+            order.getInputs().add(new MutableObject<>(op));
             order.setExecutionMode(ExecutionMode.LOCAL);
             context.computeAndSetTypeEnvironmentForOperator(order);
         }
@@ -1196,7 +1216,7 @@
         } else if (sortPrimaryKeys) {
             primaryIndexUnnestMapOp.getInputs().add(new MutableObject<>(order));
         } else {
-            primaryIndexUnnestMapOp.getInputs().add(new MutableObject<>(inputOp));
+            primaryIndexUnnestMapOp.getInputs().add(new MutableObject<>(op));
         }
         context.computeAndSetTypeEnvironmentForOperator(primaryIndexUnnestMapOp);
         primaryIndexUnnestMapOp.setExecutionMode(ExecutionMode.PARTITIONED);
@@ -1831,9 +1851,12 @@
                 joinPKVars = Collections.emptyList();
             }
 
-            return createFinalNonIndexOnlySearchPlan(dataset, inputOp, context, !isArrayIndex && sortPrimaryKeys,
-                    retainInput, retainMissing, requiresBroadcast, isArrayIndex, pkVarsFromSIdxUnnestMapOp,
-                    primaryIndexUnnestVars, joinPKVars, primaryIndexOutputTypes, leftOuterMissingValue);
+            return createFinalNonIndexOnlySearchPlan(afterTopOpRefs, dataset, inputOp, context,
+                    !isArrayIndex && sortPrimaryKeys, retainInput, retainMissing, requiresBroadcast, isArrayIndex,
+                    pkVarsFromSIdxUnnestMapOp, primaryIndexUnnestVars, joinPKVars, primaryIndexOutputTypes,
+                    leftOuterMissingValue, recordType, metaRecordType, indexSubTree, secondaryIndex, topOpRef,
+                    assignsBeforeTopOpRef, conditionRef, newMissingPlaceHolderForLOJ);
+
         } else if (!isArrayIndex) {
             // Index-only plan case: creates a UNIONALL operator that has two paths after the secondary unnest-map op,
             // and returns it.
@@ -3086,7 +3109,6 @@
                     }
                 }
 
-                // Otherwise... recurse.
                 Pair<List<String>, Integer> parentFieldNames =
                         !isIndexOnFunction
                                 ? getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, assignAndExpressionIndexes[0],
@@ -3337,4 +3359,155 @@
         }
         return isDataSourceVariableFound;
     }
+
+    private static void splitIntoConjuncts(ILogicalExpression condition, List<Mutable<ILogicalExpression>> conjuncts) {
+        if (condition.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return;
+        }
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) condition;
+        if (funcExpr.getFunctionIdentifier() == BuiltinFunctions.AND) {
+            for (Mutable<ILogicalExpression> arg : funcExpr.getArguments()) {
+                splitIntoConjuncts(arg.getValue(), conjuncts);
+            }
+        } else {
+            conjuncts.add(new MutableObject<>(condition));
+        }
+    }
+
+    protected static ILogicalExpression filterCondition(ILogicalExpression condition,
+            Set<LogicalVariable> listOfAcceptableVAR) {
+        List<Mutable<ILogicalExpression>> conjuncts = new ArrayList<>();
+        List<Mutable<ILogicalExpression>> filteredConjuncts = new ArrayList<>();
+        splitIntoConjuncts(condition, conjuncts);
+        for (Mutable<ILogicalExpression> conjunct : conjuncts) {
+            if (containsOnlyAllowedVariables(conjunct.getValue(), listOfAcceptableVAR)) {
+                filteredConjuncts.add(conjunct);
+            }
+        }
+
+        if (filteredConjuncts.size() == 1) {
+            return filteredConjuncts.get(0).getValue();
+        } else if (!filteredConjuncts.isEmpty()) {
+            ScalarFunctionCallExpression combinedCondition =
+                    new ScalarFunctionCallExpression(BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.AND));
+            combinedCondition.getArguments().addAll(filteredConjuncts);
+            combinedCondition.setSourceLocation(condition.getSourceLocation());
+            return combinedCondition;
+        } else {
+            return null;
+        }
+    }
+
+    private static boolean containsOnlyAllowedVariables(ILogicalExpression expression,
+            Set<LogicalVariable> allowedVars) {
+
+        Set<LogicalVariable> usedVars = new HashSet<>();
+        expression.getUsedVariables(usedVars);
+        for (LogicalVariable var : usedVars) {
+            if (!allowedVars.contains(var)) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    private static ILogicalOperator additionalSelectForHeterogeneousIndex(List<List<String>> chosenIndexFieldNames,
+            List<Mutable<ILogicalOperator>> afterTopOpRefs, Dataset dataset, ILogicalOperator inputOp,
+            IOptimizationContext context, boolean retainMissing, IAlgebricksConstantValue leftOuterMissingValue,
+            ARecordType recordType, ARecordType metaRecordType, OptimizableOperatorSubTree subTree,
+            Index secondaryIndex, Mutable<ILogicalOperator> topOpRef,
+            List<Mutable<ILogicalOperator>> assignsBeforeTopOpRef, Mutable<ILogicalExpression> conditionRef,
+            LogicalVariable newMissingPlaceHolderForLOJ) throws AlgebricksException {
+        List<LogicalVariable> skVarsFromSIdxUnnestMap = AccessMethodUtils.getKeyVarsFromSecondaryUnnestMap(dataset,
+                recordType, metaRecordType, inputOp, secondaryIndex, SecondaryUnnestMapOutputVarType.SECONDARY_KEY);
+
+        List<LogicalVariable> usedVarsInConditionOp = new ArrayList<>();
+        // Constructs the variable mapping between newly constructed secondary
+        // key search (SK, PK) and those in the original plan (datasource scan).
+        LinkedHashMap<LogicalVariable, LogicalVariable> origVarToSIdxUnnestMapOpVarMap = new LinkedHashMap<>();
+        VariableUtilities.getUsedVariables(topOpRef.getValue(), usedVarsInConditionOp);
+        // Gets all variables from the right (inner) branch.
+        List<LogicalVariable> liveVarsInInnerBranch = new ArrayList<>();
+        VariableUtilities.getLiveVariables(subTree.getRootRef().getValue(), liveVarsInInnerBranch);
+        for (Iterator<LogicalVariable> iterator = usedVarsInConditionOp.iterator(); iterator.hasNext();) {
+            LogicalVariable v = iterator.next();
+            if (!liveVarsInInnerBranch.contains(v)) {
+                iterator.remove();
+            }
+        }
+        List<LogicalVariable> uniqueUsedVarsInConditionOp = new ArrayList<>();
+        copyVarsToAnotherList(usedVarsInConditionOp, uniqueUsedVarsInConditionOp);
+
+        List<LogicalVariable> producedVarsInAssignsBeforeCondtionOp = new ArrayList<>();
+        List<LogicalVariable> varsTmpList = new ArrayList<>();
+        if (assignsBeforeTopOpRef != null && !assignsBeforeTopOpRef.isEmpty()) {
+            for (int i = 0; i < assignsBeforeTopOpRef.size(); i++) {
+                ILogicalOperator assignBeforeTopOp = assignsBeforeTopOpRef.get(i).getValue();
+                varsTmpList.clear();
+                VariableUtilities.getProducedVariables(assignBeforeTopOp, varsTmpList);
+                copyVarsToAnotherList(varsTmpList, producedVarsInAssignsBeforeCondtionOp);
+            }
+        }
+        List<LogicalVariable> usedVarsAfterSelect = new ArrayList<>();
+        HashSet<LogicalVariable> varsTmpSet = new HashSet<>();
+        if (afterTopOpRefs != null) {
+            for (Mutable<ILogicalOperator> afterTopOpRef : afterTopOpRefs) {
+                varsTmpSet.clear();
+                OperatorPropertiesUtil.getFreeVariablesInOp(afterTopOpRef.getValue(), varsTmpSet);
+                copyVarsToAnotherList(varsTmpSet, usedVarsAfterSelect);
+            }
+        }
+
+        for (LogicalVariable tVar : usedVarsAfterSelect) {
+
+            int sIndexIdx = chosenIndexFieldNames.indexOf(subTree.getVarsToFieldNameMap().get(tVar));
+            if (sIndexIdx == -1) {
+                continue;
+            }
+            // Constructs the mapping between the PK from the original data-scan to the PK
+            // from the secondary index search since they are different logical variables.
+            origVarToSIdxUnnestMapOpVarMap.put(tVar, skVarsFromSIdxUnnestMap.get(sIndexIdx));
+        }
+
+        List<LogicalVariable> varsUsedInTopOpButNotAfterwards = new ArrayList<>();
+        copyVarsToAnotherList(uniqueUsedVarsInConditionOp, varsUsedInTopOpButNotAfterwards);
+        varsUsedInTopOpButNotAfterwards.removeAll(usedVarsAfterSelect);
+        // For B-Tree case: if the given secondary key field variable is used only in the select or
+        // join condition, we were not able to catch the mapping between the SK from the original
+        // data-scan and the SK from the secondary index search since they are different logical variables.
+        // (E.g., we are sending a query on a composite index but returns only one field.)
+        for (LogicalVariable v : varsUsedInTopOpButNotAfterwards) {
+            int sIndexIdx = chosenIndexFieldNames.indexOf(subTree.getVarsToFieldNameMap().get(v));
+            if (sIndexIdx == -1) {
+                continue;
+            }
+            origVarToSIdxUnnestMapOpVarMap.put(v, skVarsFromSIdxUnnestMap.get(sIndexIdx));
+        }
+
+        // The additional select which will be added after the secondary index search
+        // only the conditions which include the variables used before or in secondary index search
+        // any other predicates which includes other variables should not be added here.
+        List<LogicalVariable> usedVarInInput = new ArrayList<>();
+        VariableUtilities.getUsedVariables(inputOp, usedVarInInput);
+        Set<LogicalVariable> skAcceptableVars = new HashSet<>();
+        skAcceptableVars.addAll(origVarToSIdxUnnestMapOpVarMap.keySet());
+        skAcceptableVars.addAll(usedVarInInput);
+        ILogicalExpression conditionRefExpr =
+                filterCondition(conditionRef.getValue().cloneExpression(), skAcceptableVars);
+        if (conditionRefExpr != null) {
+            LogicalVariable newMissingPlaceHolderVar = null;
+            SelectOperator newSelectOp =
+                    retainMissing
+                            ? new SelectOperator(new MutableObject<>(conditionRefExpr), leftOuterMissingValue,
+                                    newMissingPlaceHolderVar)
+                            : new SelectOperator(new MutableObject<>(conditionRefExpr));
+            newSelectOp.setSourceLocation(conditionRefExpr.getSourceLocation());
+            newSelectOp.getInputs().add(new MutableObject(inputOp));
+            VariableUtilities.substituteVariables(newSelectOp, origVarToSIdxUnnestMapOpVarMap, context);
+            newSelectOp.setExecutionMode(ExecutionMode.PARTITIONED);
+            context.computeAndSetTypeEnvironmentForOperator(newSelectOp);
+            return newSelectOp;
+        }
+        return null;
+    }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/EnumerateJoinsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/EnumerateJoinsRule.java
index 4a97acb..8b05030 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/EnumerateJoinsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/EnumerateJoinsRule.java
@@ -28,7 +28,9 @@
 
 import org.apache.asterix.common.annotations.IndexedNLJoinExpressionAnnotation;
 import org.apache.asterix.common.annotations.SkipSecondaryIndexSearchExpressionAnnotation;
+import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.translator.SqlppExpressionToPlanTranslator;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableBoolean;
 import org.apache.commons.lang3.mutable.MutableInt;
@@ -49,6 +51,8 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.HashJoinExpressionAnnotation;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
@@ -56,7 +60,9 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
 import org.apache.hyracks.algebricks.core.algebra.prettyprint.IPlanPrettyPrinter;
@@ -78,12 +84,15 @@
     private List<JoinOperator> allJoinOps; // can be inner join or left outer join
     // Will be in the order of the from clause. Important for position ordering when assigning bits to join expressions.
     private List<ILogicalOperator> leafInputs;
+    private List<Pair<ILogicalOperator, Integer>> parentsOfLeafInputs;
     private HashMap<LogicalVariable, Integer> varLeafInputIds;
     private List<Triple<Integer, Integer, Boolean>> buildSets; // the first is the bits and the second is the number of tables.
     private List<Quadruple<Integer, Integer, JoinOperator, Integer>> outerJoinsDependencyList;
     private List<AssignOperator> assignOps;
     private List<ILogicalExpression> assignJoinExprs; // These are the join expressions below the assign operator.
 
+    // for the Array UNNEST optimization. The main list is for each leafInput.
+    private List<List<List<ILogicalOperator>>> unnestOpsInfo;
     // The Distinct operators for each DataSourceScan operator (if applicable)
     private HashMap<DataSourceScanOperator, ILogicalOperator> dataScanAndGroupByDistinctOps;
 
@@ -94,6 +103,12 @@
     private ILogicalOperator rootOrderByOp;
 
     private List<LogicalVariable> resultAndJoinVars = new ArrayList();
+    private final List<Boolean> realLeafInputs = new ArrayList();
+    private boolean arrayUnnestPossible = true;
+    private int numberOfFromTerms;
+
+    private List<Triple<ILogicalOperator, ILogicalOperator, List<ILogicalOperator>>> modifyUnnestInfo;
+    private final Map<DataSourceScanOperator, Boolean> fakeLeafInputsMap = new HashMap();
 
     public EnumerateJoinsRule(JoinEnum joinEnum) {
         this.joinEnum = joinEnum;
@@ -156,22 +171,62 @@
             return false;
         }
 
-        //joinOps = new ArrayList<>();
-        allJoinOps = new ArrayList<>();
-        newJoinOps = new ArrayList<>();
-        leafInputs = new ArrayList<>();
-        varLeafInputIds = new HashMap<>();
-        outerJoinsDependencyList = new ArrayList<>();
-        assignOps = new ArrayList<>();
-        assignJoinExprs = new ArrayList<>();
-        buildSets = new ArrayList<>();
         IPlanPrettyPrinter pp = context.getPrettyPrinter();
+
+        String viewInPlan;
+        if (LOGGER.isTraceEnabled()) {
+            viewInPlan = new ALogicalPlanImpl(opRef).toString(); //useful when debugging
+        }
         printPlan(pp, (AbstractLogicalOperator) op, "Original Whole plan1");
-        leafInputNumber = 0;
-        boolean canTransform = getJoinOpsAndLeafInputs(op);
+
+        int phase = 1;
+        init(phase);
+        boolean canTransform = getJoinOpsAndLeafInputs(null, op, -1, phase);
 
         if (!canTransform) {
-            return false;
+            return cleanUp();
+        }
+
+        if (everyLeafInputDoesNotHaveADataScanOperator(leafInputs)) {
+            return cleanUp();
+        }
+        if (LOGGER.isTraceEnabled()) {
+            viewInPlan = new ALogicalPlanImpl(opRef).toString(); //useful when debugging
+        }
+        if (arrayUnnestPossible) {
+            joinEnum.stats = new Stats(context, joinEnum);
+            if (cboMode) {
+                if (!doAllDataSourcesHaveSamples(leafInputs, context)) {
+                    return cleanUp();
+                }
+            }
+            // Here on, we expect that changes can be made to the incoming plan and that optimization will proceed
+            // without any hitch. Basically, we cannot go back now!!
+            // now that we know it is safe to proceed with unnesting array optimization, we will remove
+            // the unnestOps and related assign ops from the leafInputs and add them back later at the right places.
+            int i = -1;
+            int j = -1;
+            for (List<List<ILogicalOperator>> l : unnestOpsInfo) {
+                i++;
+                if (realLeafInputs.get(i)) {
+                    j++;
+                    removeUnnestOpsFromLeafInputLevel1(leafInputs.get(j), l);
+                }
+            }
+
+            // now the plan should have no unnestOps and no related assigns
+            if (LOGGER.isTraceEnabled()) {
+                String viewOldPlan = new ALogicalPlanImpl(opRef).toString(); //useful when debugging
+            }
+            introduceFakeOuterJoins(opRef, context);
+            if (LOGGER.isTraceEnabled()) {
+                String viewNewPlan = new ALogicalPlanImpl(opRef).toString(); //useful when debugging
+            }
+            phase = 2;
+            init(phase);
+            getJoinOpsAndLeafInputs(null, op, -1, phase);
+        } else {
+            unnestOpsInfo.clear();
         }
 
         collectJoinConditionsVariables(); // will be used for determining which variables will be projected from the base levels
@@ -179,10 +234,10 @@
         convertOuterJoinstoJoinsIfPossible(outerJoinsDependencyList);
 
         printPlan(pp, (AbstractLogicalOperator) op, "Original Whole plan2");
-        int numberOfFromTerms = leafInputs.size();
+        numberOfFromTerms = leafInputs.size();
 
         if (LOGGER.isTraceEnabled()) {
-            String viewInPlan = new ALogicalPlanImpl(opRef).toString(); //useful when debugging
+            viewInPlan = new ALogicalPlanImpl(opRef).toString(); //useful when debugging
             LOGGER.trace("viewInPlan");
             LOGGER.trace(viewInPlan);
         }
@@ -192,14 +247,18 @@
             // we need to build the smaller sets first. So we need to find these first.
         }
         joinEnum.initEnum((AbstractLogicalOperator) op, cboMode, cboTestMode, numberOfFromTerms, leafInputs, allJoinOps,
-                assignOps, outerJoinsDependencyList, buildSets, varLeafInputIds, dataScanAndGroupByDistinctOps,
-                rootGroupByDistinctOp, rootOrderByOp, resultAndJoinVars, context);
+                assignOps, outerJoinsDependencyList, buildSets, varLeafInputIds, unnestOpsInfo,
+                dataScanAndGroupByDistinctOps, rootGroupByDistinctOp, rootOrderByOp, resultAndJoinVars,
+                fakeLeafInputsMap, context);
 
         if (cboMode) {
             if (!doAllDataSourcesHaveSamples(leafInputs, context)) {
-                return false;
+                return cleanUp();
             }
         }
+        if (LOGGER.isTraceEnabled()) {
+            viewInPlan = new ALogicalPlanImpl(opRef).toString(); //useful when debugging
+        }
 
         printLeafPlans(pp, leafInputs, "Inputs1");
 
@@ -207,32 +266,59 @@
             pushAssignsIntoLeafInputs(pp, leafInputs, assignOps, assignJoinExprs);
         }
 
+        if (LOGGER.isTraceEnabled()) {
+            viewInPlan = new ALogicalPlanImpl(opRef).toString(); //useful when debugging
+        }
         printLeafPlans(pp, leafInputs, "Inputs2");
-
+        if (LOGGER.isTraceEnabled()) {
+            String viewPlan = new ALogicalPlanImpl(opRef).toString(); //useful when debugging
+        }
         int cheapestPlan = joinEnum.enumerateJoins(); // MAIN CALL INTO CBO
         if (cheapestPlan == PlanNode.NO_PLAN) {
-            return false;
+            return cleanUp();
         }
 
         PlanNode cheapestPlanNode = joinEnum.allPlans.get(cheapestPlan);
 
         generateHintWarnings();
-
+        ILogicalOperator root = op;
         if (numberOfFromTerms > 1) {
             getNewJoinOps(cheapestPlanNode, allJoinOps);
             if (allJoinOps.size() != newJoinOps.size()) {
-                return false; // there are some cases such as R OJ S on true. Here there is an OJ predicate but the code in findJoinConditions
+                return cleanUp(); // there are some cases such as R OJ S on true. Here there is an OJ predicate but the code in findJoinConditions
                 // in JoinEnum does not capture this. Will fix later. Just bail for now.
             }
+            if (LOGGER.isTraceEnabled()) {
+                String viewInPlan2 = new ALogicalPlanImpl(opRef).toString(); //useful when debugging
+            }
             buildNewTree(cheapestPlanNode, newJoinOps, new MutableInt(0), context);
-            opRef.setValue(newJoinOps.get(0));
+            root = newJoinOps.get(0);
+            if (LOGGER.isTraceEnabled()) {
+                String viewInPlan2 = new ALogicalPlanImpl(new MutableObject<>(root)).toString();
+            }
+            if (phase == 2) {
+                // Now remove the Fake outer joins and put in the original Unnest Ops along with the corresponding Assign Ops
+                modifyUnnestInfo = new ArrayList<>();
+                collectUnnestModificationInfo(null, root, cheapestPlanNode);
+                for (int k = 0; k < modifyUnnestInfo.size(); k++) {
+                    root = modifyTree(null, root, k);
+                }
+                Mutable<ILogicalOperator> rootRef = new MutableObject<>(root);
+                if (LOGGER.isTraceEnabled()) {
+                    String viewInPlan2 = new ALogicalPlanImpl(rootRef).toString(); //useful when debugging
+                }
+            }
+            opRef.setValue(root);
+            if (LOGGER.isTraceEnabled()) {
+                String viewInPlan2 = new ALogicalPlanImpl(opRef).toString(); //useful when debugging
+            }
+            context.computeAndSetTypeEnvironmentForOperator(root);
 
             if (assignOps.size() > 0) {
                 for (int i = assignOps.size() - 1; i >= 0; i--) {
                     MutableBoolean removed = new MutableBoolean(false);
                     removed.setFalse();
-                    pushAssignsAboveJoins(newJoinOps.get(0), assignOps.get(i), assignJoinExprs.get(i), removed);
-                    context.computeAndSetTypeEnvironmentForOperator(newJoinOps.get(i));
+                    pushAssignsAboveJoins(root, assignOps.get(i), assignJoinExprs.get(i), removed);
                     context.computeAndSetTypeEnvironmentForOperator(assignOps.get(i));
                     if (removed.isTrue()) {
                         assignOps.remove(i);
@@ -240,9 +326,9 @@
                 }
             }
 
-            printPlan(pp, (AbstractLogicalOperator) newJoinOps.get(0), "New Whole Plan after buildNewTree 1");
-            ILogicalOperator root = addRemainingAssignsAtTheTop(newJoinOps.get(0), assignOps);
-            printPlan(pp, (AbstractLogicalOperator) newJoinOps.get(0), "New Whole Plan after buildNewTree 2");
+            printPlan(pp, (AbstractLogicalOperator) root, "New Whole Plan after buildNewTree 1");
+            root = addRemainingAssignsAtTheTop(root, assignOps);
+            printPlan(pp, (AbstractLogicalOperator) root, "New Whole Plan after buildNewTree 2");
             printPlan(pp, (AbstractLogicalOperator) root, "New Whole Plan after buildNewTree");
 
             // this will be the new root
@@ -257,11 +343,6 @@
             if (LOGGER.isTraceEnabled()) {
                 LOGGER.trace("---------------------------- Printing Leaf Inputs");
                 printLeafPlans(pp, leafInputs, "Inputs");
-                // print joins starting from the bottom
-                for (int i = newJoinOps.size() - 1; i >= 0; i--) {
-                    printPlan(pp, (AbstractLogicalOperator) newJoinOps.get(i), "join " + i);
-                }
-                printPlan(pp, (AbstractLogicalOperator) newJoinOps.get(0), "New Whole Plan");
                 printPlan(pp, (AbstractLogicalOperator) root, "New Whole Plan");
             }
             // turn of this rule for all joins in this set (subtree)
@@ -271,10 +352,313 @@
         } else {
             buildNewTree(cheapestPlanNode);
         }
-        context.computeAndSetTypeEnvironmentForOperator(op);
+        context.computeAndSetTypeEnvironmentForOperator(root);
+        String finalPlan = new ALogicalPlanImpl(opRef).toString(); //useful when debugging
         return true;
     }
 
+    private boolean everyLeafInputDoesNotHaveADataScanOperator(List<ILogicalOperator> leafInputs) {
+        for (ILogicalOperator leafInput : leafInputs) {
+            DataSourceScanOperator scanOp = (DataSourceScanOperator) findDataSourceScanOperator(leafInput);
+            if (scanOp == null) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    private boolean cleanUp() {
+        removeTrueFromAllLeafInputs();
+        return false;
+    }
+
+    private ILogicalOperator modifyTree(ILogicalOperator parent, ILogicalOperator op, int k) {
+
+        if (modifyUnnestInfo.get(k).second == op) { // found the one to get rid off; this should be an OJ
+            int size = modifyUnnestInfo.get(k).third.size();
+            modifyUnnestInfo.get(k).third.get(size - 1).getInputs().get(0).setValue(op.getInputs().get(0).getValue()); //The unnestOp is at the lowest position and points to the op below it.
+            if (parent == null) {
+                ILogicalOperator root = modifyUnnestInfo.get(k).third.get(0); // the first assign that belongs to this unnestOp. Or an only one UnnestOp with no assigns
+                ILogicalOperator q = root;
+                if (modifyUnnestInfo.get(k).third.size() > 1) {
+                    for (ILogicalOperator p : modifyUnnestInfo.get(k).third) {
+                        q.getInputs().get(0).setValue(p);
+                        q = p;
+                    }
+                }
+                return root;
+            } else {
+                ILogicalOperator q = parent;
+                for (ILogicalOperator p : modifyUnnestInfo.get(k).third) {
+                    q.getInputs().get(0).setValue(p);
+                    q = p;
+                }
+                return parent;
+            }
+
+        }
+        for (Mutable<ILogicalOperator> input : op.getInputs()) {
+            return modifyTree(op, input.getValue(), k);
+        }
+        return null;
+    }
+
+    // This is a complicated routine. Removes unnestOperations from leafInputs. They will be added back at the right places.
+    // Replaces fakeOuterJoins with Unnest Operations
+    // The idea is very simple. We replace an UnnestOp (input a) with a LOJ (inputs a, fake datasource Scan Op b).
+    // This goes to CBO.  When CBO returns, the LOJ's move around.
+    // We replace the LOJ (input x, input b) with the UnnestOp (x)
+    private void collectUnnestModificationInfo(ILogicalOperator parent, ILogicalOperator op, PlanNode plan) {
+        // We cant go by the old leafInputs and parent structures, since the leafInputs may be in different places now.
+
+        if (joinClause(op)) {
+            PlanNode left = plan.getLeftPlanNode();
+            PlanNode right = plan.getRightPlanNode();
+            int rightjnNum = plan.getRightJoinIndex();
+            JoinNode rightjn = joinEnum.jnArray[rightjnNum];
+            if (rightjn.getFake()) {
+                int leafInputNumber = rightjn.getLeafInputNumber();
+                int arrayRef = rightjn.getArrayRef();
+                modifyUnnestInfo
+                        .add(new Triple<>(parent, op, unnestOpsInfo.get(leafInputNumber - 1).get(arrayRef - 1)));
+            }
+            parent = op;
+            collectUnnestModificationInfo(parent, op.getInputs().get(0).getValue(), left);
+            collectUnnestModificationInfo(parent, op.getInputs().get(1).getValue(), right);
+        }
+    }
+
+    // create one fake outer join for each unnest operation;
+    private void introduceFakeOuterJoins(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        int i = -1;
+        int j = -1;
+
+        for (List<List<ILogicalOperator>> l1 : unnestOpsInfo) {// each loop here is for a particular leafInout
+            i++;
+            if (realLeafInputs.get(i)) {
+                j++;
+            }
+            if (l1.size() == 0) {
+                continue;
+            }
+
+            LeftOuterJoinOperator foj = null;
+            ILogicalOperator leftChild = leafInputs.get(j);
+            ILogicalOperator leafInput = leafInputs.get(j);
+            // We will add as many left outer joins as there are elements in l1
+            // We will not modify leafInput. We will do that before issuing sampling queries. REMOVE
+            ILogicalOperator parentOp = null; // The final left outerjoin operator is what we will attach the leafInput to
+
+            for (List<ILogicalOperator> l2 : l1) {
+                String viewInPlan = new ALogicalPlanImpl(opRef).toString(); //useful when debugging
+                DataSourceScanOperator fakeDs = (DataSourceScanOperator) truncateInput(leafInput);
+                fakeLeafInputsMap.put(fakeDs, true);
+                LogicalVariable var1 = fakeDs.getVariables().get(0);
+                MutableObject<ILogicalOperator> q = new MutableObject<>(fakeDs);
+                LogicalVariable var2 = modify(q.getValue(), context); // so as to make it fake, remove teh original variables
+                ILogicalExpression expr = makeNewexpr(var1, var2);
+                foj = new LeftOuterJoinOperator(new MutableObject<>(expr), new MutableObject<>(leftChild), q,
+                        ConstantExpression.MISSING.getValue());
+                viewInPlan = new ALogicalPlanImpl(opRef).toString(); //useful when debugging
+                leftChild = foj;
+                viewInPlan = new ALogicalPlanImpl(opRef).toString(); //useful when debugging
+            }
+            Pair<ILogicalOperator, Integer> parent = parentsOfLeafInputs.get(i);
+            parent.first.getInputs().get(parent.second).setValue(foj);
+            String viewInPlan = new ALogicalPlanImpl(opRef).toString(); //useful when debugging
+        }
+    }
+
+    private ILogicalExpression makeNewexpr(LogicalVariable var1, LogicalVariable var2) {
+        List<Mutable<ILogicalExpression>> arguments = new ArrayList<>();
+        VariableReferenceExpression e1 = new VariableReferenceExpression(var1);
+        arguments.add(new MutableObject<>(e1));
+        VariableReferenceExpression e2 = new VariableReferenceExpression(var2);
+        arguments.add(new MutableObject<>(e2));
+        ScalarFunctionCallExpression expr = new ScalarFunctionCallExpression(
+                FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.EQ), arguments);
+        return expr;
+    }
+
+    // remove the old variables and add a new variable.
+    private LogicalVariable modify(ILogicalOperator op, IOptimizationContext context) {
+        DataSourceScanOperator dsOp = (DataSourceScanOperator) op;
+        int s = dsOp.getVariables().size();
+        dsOp.getVariables().clear();
+        for (int i = 0; i < s; i++) {
+            LogicalVariable newVar = context.newVar(); // need all three for alias stuff
+            dsOp.getVariables().add(newVar);
+        }
+        return dsOp.getVariables().get(0);
+    }
+
+    private ILogicalOperator truncateInput(ILogicalOperator op) throws AlgebricksException {
+        ILogicalOperator dsOp = findDataSourceScanOperator(op);
+        ILogicalOperator ds = OperatorManipulationUtil.bottomUpCopyOperators(dsOp);
+        return ds;
+    }
+
+    private void init(int phase) {
+        allJoinOps = new ArrayList<>();
+        newJoinOps = new ArrayList<>();
+        leafInputs = new ArrayList<>();
+        varLeafInputIds = new HashMap<>();
+        if (phase == 1)
+            unnestOpsInfo = new ArrayList<>();
+        outerJoinsDependencyList = new ArrayList<>();
+        parentsOfLeafInputs = new ArrayList<>();
+        assignOps = new ArrayList<>();
+        assignJoinExprs = new ArrayList<>();
+        buildSets = new ArrayList<>();
+        leafInputNumber = 0;
+    }
+
+    private void findUnnestOps(ILogicalOperator leafInput) throws AlgebricksException {
+        ILogicalOperator p = leafInput;
+        List<ILogicalOperator> unnestOps = findAllUnnestOps(p); // how many and which ones
+        for (ILogicalOperator op : unnestOps) {
+            UnnestOperator unnestOp = (UnnestOperator) op;
+            if (anyVarIsAJoinVar(unnestOp.getVariables())) {
+                unnestOpsInfo.add(new ArrayList<>()); // each leafInput must have one entry
+                arrayUnnestPossible = false; // If these variables participate in join predicates, then unnestOps cannot be moved above joins
+            }
+        }
+        List<List<ILogicalOperator>> bigList = new ArrayList<>();
+        realLeafInputs.add(true);
+        for (int i = 0; i < unnestOps.size(); i++) {
+            List<ILogicalOperator> ops = new ArrayList<>(); //Gather all AssignsOps, if any, associated wth this unnestOp
+            UnnestOperator unnestOp = (UnnestOperator) unnestOps.get(i);
+
+            while (p != null && p.getOperatorTag() != LogicalOperatorTag.EMPTYTUPLESOURCE) {
+                if (p.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+                    AssignOperator aOp = (AssignOperator) p;
+
+                    ILogicalExpression a = aOp.getExpressions().get(0).getValue();
+                    if (a.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                        AbstractFunctionCallExpression exp =
+                                (AbstractFunctionCallExpression) aOp.getExpressions().get(0).getValue();
+                        if (exp.getKind() == AbstractFunctionCallExpression.FunctionKind.SCALAR) {
+                            ILogicalExpression lexp = exp.getArguments().get(0).getValue();
+                            if (lexp.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+                                VariableReferenceExpression varRef = (VariableReferenceExpression) lexp;
+                                LogicalVariable var = varRef.getVariableReference();
+                                LogicalVariable unnestVar = unnestOp.getVariables().get(0);
+                                if (unnestVar == var) {
+                                    if ((anyVarIsAJoinVar(aOp.getVariables())
+                                            || assignVarPresentInLeafInput(aOp, leafInput))) {
+                                        unnestOpsInfo.add(new ArrayList<>());
+                                        arrayUnnestPossible = false;
+                                    } else {
+                                        ops.add(aOp);
+                                    }
+                                }
+                            }
+                        }
+                    }
+                }
+                p = p.getInputs().get(0).getValue();
+            }
+            ops.add(unnestOp); // the unnestOp will be the last (and may be the only op)
+            bigList.add(ops);
+        }
+        unnestOpsInfo.add(bigList); // one for each LeafInput. If empty, means that there are no array references in this leafInout
+        // also need to add some dummy entries for the fake leafInputs. Add as many as unnestOps. This will make the code in setCardsAndSizes happy
+
+        for (ILogicalOperator q : unnestOps) {
+            bigList = new ArrayList<>();
+            unnestOpsInfo.add(bigList);
+            realLeafInputs.add(false);
+        }
+    }
+
+    private boolean assignVarPresentInLeafInput(AssignOperator aOp, ILogicalOperator leafInput)
+            throws AlgebricksException {
+        List<LogicalVariable> vars = new ArrayList<>();
+        for (LogicalVariable var : aOp.getVariables()) {
+            ILogicalOperator p = leafInput;
+            while (true) {
+                vars.clear();
+                VariableUtilities.getUsedVariables(p, vars);
+                if (vars.contains(var)) {
+                    return true;
+                }
+                if (p == aOp) { // No need to go below the assignOp
+                    break;
+                }
+                p = p.getInputs().get(0).getValue();
+            }
+        }
+        return false;
+    }
+
+    private boolean anyVarIsAJoinVar(List<LogicalVariable> vars) {
+        for (LogicalVariable var : vars) {
+            if (varIsAJoinVar(var)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    private boolean varIsAJoinVar(LogicalVariable var) {
+        for (JoinOperator j : allJoinOps) {
+            List<LogicalVariable> joinExprVars = new ArrayList<>();
+            AbstractBinaryJoinOperator jo = j.getAbstractJoinOp();
+            ILogicalExpression expr = jo.getCondition().getValue();
+            joinExprVars.clear();
+            expr.getUsedVariables(joinExprVars);
+            for (LogicalVariable lv : joinExprVars) {
+                if (lv.equals(var)) {
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+
+    // Guessing this is inefficient but do not expect leafInputs to be huge, so efficiency may not be a concern
+    private static void removeUnnestOpsFromLeafInputLevel1(ILogicalOperator leafInput,
+            List<List<ILogicalOperator>> bigList) {
+        for (List<ILogicalOperator> l : bigList) {
+            removeUnnestOpsFromLeafInputLevel2(leafInput, l);
+        }
+    }
+
+    private static void removeUnnestOpsFromLeafInputLevel2(ILogicalOperator leafInput, List<ILogicalOperator> list) {
+        for (ILogicalOperator op : list) {
+            removeUnnestOpFromLeafInputLevel3(leafInput, op);
+        }
+    }
+
+    private static void removeUnnestOpFromLeafInputLevel3(ILogicalOperator leafInput, ILogicalOperator op) {
+        ILogicalOperator parent = leafInput; // always a select Op with condition true
+        ILogicalOperator p = leafInput.getInputs().get(0).getValue();
+
+        while (p != null && p.getOperatorTag() != LogicalOperatorTag.EMPTYTUPLESOURCE) {
+            if (p == op) {
+                parent.getInputs().get(0).setValue(op.getInputs().get(0).getValue());
+            }
+            parent = p;
+            p = p.getInputs().get(0).getValue();
+        }
+    }
+
+    private List<ILogicalOperator> findAllUnnestOps(ILogicalOperator p) {
+        List<ILogicalOperator> list = new ArrayList<>();
+        while (p != null && p.getOperatorTag() != LogicalOperatorTag.EMPTYTUPLESOURCE) {
+            if (p.getOperatorTag() == LogicalOperatorTag.UNNEST) {
+                UnnestOperator unnestOp = (UnnestOperator) p;
+                Object arrayAccess = unnestOp.getAnnotations().get(SqlppExpressionToPlanTranslator.ARRAY_ACCESS);
+                if (arrayAccess != null && (boolean) arrayAccess) {
+                    list.add(p);
+                }
+            }
+            p = p.getInputs().get(0).getValue();
+        }
+        return list;
+    }
+
     private void collectJoinConditionsVariables() {
         for (JoinOperator jOp : allJoinOps) {
             AbstractBinaryJoinOperator joinOp = jOp.getAbstractJoinOp();
@@ -295,7 +679,6 @@
 
     private void pushAssignsAboveJoins(ILogicalOperator op, AssignOperator aOp, ILogicalExpression jexpr,
             MutableBoolean removed) {
-        System.out.println("op " + op.toString());
         if (!op.getInputs().isEmpty()) {
             for (int i = 0; i < op.getInputs().size(); i++) {
                 ILogicalOperator oper = op.getInputs().get(i).getValue();
@@ -641,7 +1024,8 @@
      * join operators but switch the leafInputs (see buildNewTree). The whole scheme is based on the assumption that the
      * leafInputs can be switched. The various data structures make the leafInputs accessible efficiently.
      */
-    private boolean getJoinOpsAndLeafInputs(ILogicalOperator op) throws AlgebricksException {
+    private boolean getJoinOpsAndLeafInputs(ILogicalOperator parent, ILogicalOperator op, int leftRight, int phase)
+            throws AlgebricksException {
         if (joinClause(op)) {
             JoinOperator jO = new JoinOperator((AbstractBinaryJoinOperator) op);
             allJoinOps.add(jO);
@@ -654,7 +1038,7 @@
             for (int i = 0; i < 2; i++) {
                 ILogicalOperator nextOp = op.getInputs().get(i).getValue();
                 firstLeafInputNumber = leafInputNumber + 1; // we are interested in the 2nd input only
-                boolean canTransform = getJoinOpsAndLeafInputs(nextOp);
+                boolean canTransform = getJoinOpsAndLeafInputs(op, nextOp, i, phase);
                 if (!canTransform) {
                     return false;
                 }
@@ -694,7 +1078,19 @@
                     }
                 } else {
                     leafInputNumber++;
-                    leafInputs.add(op);
+                    // always add a SelectOperator with TRUE condition. The code below becomes simpler with a select operator.
+                    // We will always have a parent op, so we can remove the operator below this selOp without affecting the leafInput
+                    if (phase == 1) { // dont want to add two of these
+                        SelectOperator selOp = new SelectOperator(new MutableObject<>(ConstantExpression.TRUE));
+                        parent.getInputs().get(leftRight).setValue(selOp);
+                        selOp.getInputs().add(new MutableObject<>(null)); //add an input
+                        selOp.getInputs().get(0).setValue(op);
+                        leafInputs.add(selOp);
+                        findUnnestOps(selOp);
+                    } else {
+                        leafInputs.add(op);
+                    }
+                    parentsOfLeafInputs.add(new Pair<>(parent, leftRight));
                     if (!addLeafInputNumbersToVars(op)) {
                         return false;
                     }
@@ -702,7 +1098,7 @@
             } else { // This must be an internal edge
                 if (onlyAssigns(op, assignOps)) {
                     ILogicalOperator skipAssisgnsOp = skipPastAssigns(op);
-                    boolean canTransform = getJoinOpsAndLeafInputs(skipAssisgnsOp);
+                    boolean canTransform = getJoinOpsAndLeafInputs(op, skipAssisgnsOp, leftRight, phase);
                     if (!canTransform) {
                         return false;
                     }
@@ -926,8 +1322,6 @@
         List<PlanNode> allPlans = joinEnum.getAllPlans();
         int leftIndex = plan.getLeftPlanIndex();
         int rightIndex = plan.getRightPlanIndex();
-        //System.out.println("allPlansSize " + allPlans.size() + " leftIndex " + leftIndex + " rightIndex " + rightIndex); // put in trace statements
-        //System.out.println("allPlansSize " + allPlans.size());
         PlanNode leftPlan = allPlans.get(leftIndex);
         PlanNode rightPlan = allPlans.get(rightIndex);
 
@@ -939,14 +1333,15 @@
 
         if (leftPlan.IsScanNode()) {
             // leaf
-            ILogicalOperator leftInput = leftPlan.getLeafInput();
+            ILogicalOperator leftInput = removeTrue(leftPlan.getLeafInput());
             skipAllIndexes(leftPlan, leftInput);
             ILogicalOperator selOp = findSelectOrUnnestOrDataScan(leftInput);
             if (selOp != null) {
                 addCardCostAnnotations(selOp, leftPlan);
             }
             joinOp.getInputs().get(0).setValue(leftInput);
-            context.computeAndSetTypeEnvironmentForOperator(joinOp.getInputs().get(0).getValue());
+            ILogicalOperator op = joinOp.getInputs().get(0).getValue();
+            context.computeAndSetTypeEnvironmentForOperator(op);
             addCardCostAnnotations(findDataSourceScanOperator(leftInput), leftPlan);
         } else {
             // join
@@ -959,7 +1354,7 @@
 
         if (rightPlan.IsScanNode()) {
             // leaf
-            ILogicalOperator rightInput = rightPlan.getLeafInput();
+            ILogicalOperator rightInput = removeTrue(rightPlan.getLeafInput());
             skipAllIndexes(rightPlan, rightInput);
             ILogicalOperator selOp = findSelectOrUnnestOrDataScan(rightInput);
             if (selOp != null) {
@@ -978,6 +1373,30 @@
         }
     }
 
+    static ILogicalOperator removeTrue(ILogicalOperator leafInput) {
+        if (leafInput.getOperatorTag() == LogicalOperatorTag.SELECT) {
+            SelectOperator selOp = (SelectOperator) leafInput;
+            if (selOp.getCondition().getValue() == ConstantExpression.TRUE) {
+                return leafInput.getInputs().get(0).getValue();
+            }
+        }
+        return leafInput;
+    }
+
+    // remove any selectops that may have been added in phase1
+    private void removeTrueFromAllLeafInputs() {
+        for (Pair<ILogicalOperator, Integer> parent : parentsOfLeafInputs) {
+            ILogicalOperator nextOp = parent.getFirst().getInputs().get(parent.getSecond()).getValue();
+            if (nextOp.getOperatorTag() == LogicalOperatorTag.SELECT) {
+                SelectOperator selOp = (SelectOperator) nextOp;
+                if (selOp.getCondition().getValue() == ConstantExpression.TRUE) {
+                    parent.getFirst().getInputs().get(parent.getSecond())
+                            .setValue(nextOp.getInputs().get(0).getValue());
+                }
+            }
+        }
+    }
+
     // in some very rare cases, there is an internal edge that has an assign statement such as $$var = 20 but this variable
     // is not used anywhere in the current join graph but is used outside the current join graph. So we add this assign to the top of
     // our plan, so the rest of the code will be happy. Strange that this assign appears in the join graph.
@@ -1031,15 +1450,22 @@
     // check to see if every dataset has a sample. If not, CBO code cannot run. A warning message must be issued as well.
     private boolean doAllDataSourcesHaveSamples(List<ILogicalOperator> leafInputs, IOptimizationContext context)
             throws AlgebricksException {
+        int n = 0;
         for (ILogicalOperator li : leafInputs) {
             DataSourceScanOperator scanOp = (DataSourceScanOperator) findDataSourceScanOperator(li);
-            if (scanOp == null)
+            if (scanOp == null) {
                 continue;
-            Index index = joinEnum.getStatsHandle().findSampleIndex(scanOp, context);
+            }
+            Stats handle = joinEnum.getStatsHandle();
+            if (handle == null) {
+                continue;
+            }
+            Index index = handle.findSampleIndex(scanOp, context);
             if (index == null) {
                 return false;
             }
+            n++;
         }
-        return true;
+        return (leafInputs.size() == n);
     }
-}
\ No newline at end of file
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinCondition.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinCondition.java
index d56d38a..b5c290f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinCondition.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinCondition.java
@@ -29,6 +29,7 @@
     protected boolean outerJoin;
     private boolean derived = false;
     protected boolean partOfComposite = false;
+    protected boolean deleted = false;
     protected int numberOfVars = 0; // how many variables
     protected int componentNumber = 0; // for identifying if join graph is connected
     protected int datasetBits;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinEnum.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinEnum.java
index 479e395..c8675df 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinEnum.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinEnum.java
@@ -116,6 +116,7 @@
     protected List<AssignOperator> assignOps;
     List<Quadruple<Integer, Integer, JoinOperator, Integer>> outerJoinsDependencyList;
     HashMap<LogicalVariable, Integer> varLeafInputIds;
+    protected List<List<List<ILogicalOperator>>> unnestOpsInfo;
     protected List<JoinOperator> allJoinOps;
     protected ILogicalOperator localJoinOp; // used in nestedLoopsApplicable code.
     protected IOptimizationContext optCtx;
@@ -137,6 +138,7 @@
     protected ICost cost;
     protected ICostMethods costMethods;
     List<LogicalVariable> resultAndJoinVars;
+    Map<DataSourceScanOperator, Boolean> fakeLeafInputsMap;
 
     public JoinEnum() {
     }
@@ -145,9 +147,11 @@
             List<ILogicalOperator> leafInputs, List<JoinOperator> allJoinOps, List<AssignOperator> assignOps,
             List<Quadruple<Integer, Integer, JoinOperator, Integer>> outerJoinsDependencyList,
             List<Triple<Integer, Integer, Boolean>> buildSets, HashMap<LogicalVariable, Integer> varLeafInputIds,
+            List<List<List<ILogicalOperator>>> unnestOpsInfo,
             HashMap<DataSourceScanOperator, ILogicalOperator> dataScanAndGroupByDistinctOps,
             ILogicalOperator grpByDistinctOp, ILogicalOperator orderByOp, List<LogicalVariable> resultAndJoinVars,
-            IOptimizationContext context) throws AsterixException {
+            Map<DataSourceScanOperator, Boolean> fakeLeafInputsMap, IOptimizationContext context)
+            throws AsterixException {
         this.singleDatasetPreds = new ArrayList<>();
         this.joinConditions = new ArrayList<>();
         this.joinHints = new HashMap<>();
@@ -166,10 +170,12 @@
         this.allJoinOps = allJoinOps;
         this.buildSets = buildSets;
         this.varLeafInputIds = varLeafInputIds;
+        this.unnestOpsInfo = unnestOpsInfo;
         this.dataScanAndGroupByDistinctOps = dataScanAndGroupByDistinctOps;
         this.rootGroupByDistinctOp = grpByDistinctOp;
         this.rootOrderByOp = orderByOp;
         this.resultAndJoinVars = resultAndJoinVars;
+        this.fakeLeafInputsMap = fakeLeafInputsMap;
         this.op = op;
         this.forceJoinOrderMode = getForceJoinOrderMode(context);
         this.queryPlanShape = getQueryPlanShape(context);
@@ -919,6 +925,14 @@
             JoinNode jn = jnArray[i];
             Index.SampleIndexDetails idxDetails = jn.getIdxDetails();
             ILogicalOperator leafInput = this.leafInputs.get(i - 1);
+            DataSourceScanOperator scanOp = findDataSourceScanOperator(leafInput);
+            if (scanOp != null && fakeLeafInputsMap.get(scanOp) != null) {
+                jn.setFake();
+            }
+            int numArrayRefs = 0;
+            if (unnestOpsInfo.size() > 0) {
+                numArrayRefs = unnestOpsInfo.get(i - 1).size();
+            }
             if (!cboTestMode) {
                 if (idxDetails == null) {
                     dataScanPlan = jn.addSingleDatasetPlans();
@@ -927,10 +941,11 @@
                     }
                     continue;
                 }
-                jn.setCardsAndSizes(idxDetails, leafInput);
-
                 // Compute the distinct cardinalities for each base join node.
-                DataSourceScanOperator scanOp = findDataSourceScanOperator(leafInput);
+                if (!jn.getFake()) {
+                    jn.setCardsAndSizes(idxDetails, leafInput, i); // the fake case gets handled in this routine.
+                }
+
                 ILogicalOperator grpByDistinctOp = this.dataScanAndGroupByDistinctOps.get(scanOp);
                 if (grpByDistinctOp != null) {
                     long distinctCardinality = stats.findDistinctCardinality(grpByDistinctOp);
@@ -940,6 +955,11 @@
                     grpByDistinctOp.getAnnotations().put(OperatorAnnotations.OP_INPUT_CARDINALITY, grpInputCard);
                     grpByDistinctOp.getAnnotations().put(OperatorAnnotations.OP_OUTPUT_CARDINALITY, grpOutputCard);
                 }
+            } else {
+                // cboTestMode. There are no samples here.
+                for (int j = 1; j <= numArrayRefs; j++) {
+                    jn.setCardsAndSizesForFakeJn(i, j, 10.0);
+                }
             }
 
             dataScanPlan = jn.addSingleDatasetPlans();
@@ -947,7 +967,7 @@
                 return PlanNode.NO_PLAN;
             }
             // We may not add any index plans, so need to check for NO_PLAN
-            jn.addIndexAccessPlans(leafInput);
+            jn.addIndexAccessPlans(EnumerateJoinsRule.removeTrue(leafInput));
         }
         return this.numberOfTerms;
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinNode.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinNode.java
index 2c430b1..bc4709e 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinNode.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinNode.java
@@ -116,6 +116,9 @@
     private double sizeVarsFromDisk = -1.0;
     private double sizeVarsAfterScan = -1.0;
     private boolean columnar = true; // default
+    private boolean fake = false; // default; Fake will be set to true when we introduce fake leafInputs for unnesting arrays.
+    private int leafInputNumber = -1; // this field and the next are used for Array Unnest ops.
+    private int arrayRef = -1;
 
     private JoinNode(int i) {
         this.jnArrayIndex = i;
@@ -249,12 +252,52 @@
         return columnar;
     }
 
-    public void setCardsAndSizes(Index.SampleIndexDetails idxDetails, ILogicalOperator leafInput)
+    public void setFake() {
+        fake = true;
+    }
+
+    public boolean getFake() {
+        return fake;
+    }
+
+    public void setLeafInputNumber(int inputNumber) {
+        leafInputNumber = inputNumber;
+    }
+
+    public int getLeafInputNumber() {
+        return leafInputNumber;
+    }
+
+    public void setArrayRef(int arrayRef) {
+        this.arrayRef = arrayRef;
+    }
+
+    public int getArrayRef() {
+        return arrayRef;
+    }
+
+    protected void setCardsAndSizesForFakeJn(int leafInputNumber, int arrayRef, double unnestFactor) {
+        joinEnum.jnArray[leafInputNumber + arrayRef].setOrigCardinality(unnestFactor, false);
+        joinEnum.jnArray[leafInputNumber + arrayRef].setCardinality(unnestFactor, false);
+        joinEnum.jnArray[leafInputNumber + arrayRef].setSizeVarsFromDisk(4);
+        joinEnum.jnArray[leafInputNumber + arrayRef].setSizeVarsAfterScan(4);
+        joinEnum.jnArray[leafInputNumber + arrayRef].setAvgDocSize(4);
+        joinEnum.jnArray[leafInputNumber + arrayRef].setLeafInputNumber(leafInputNumber);
+        joinEnum.jnArray[leafInputNumber + arrayRef].setArrayRef(arrayRef);
+    }
+
+    public void setCardsAndSizes(Index.SampleIndexDetails idxDetails, ILogicalOperator leafInput, int leafInputNumber)
             throws AlgebricksException {
 
+        //double origDatasetCard, finalDatasetCard, sampleCard1, sampleCard2;
         double origDatasetCard, finalDatasetCard, sampleCard;
         unnestFactor = 1.0;
 
+        int numArrayRefs = 0;
+        if (joinEnum.unnestOpsInfo.size() > 0) {
+            numArrayRefs = joinEnum.unnestOpsInfo.get(leafInputNumber - 1).size();
+        }
+
         DataSourceScanOperator scanOp = joinEnum.findDataSourceScanOperator(leafInput);
         if (scanOp == null) {
             return; // what happens to the cards and sizes then? this may happen in case of in lists
@@ -262,12 +305,15 @@
 
         List<List<IAObject>> result;
         SelectOperator selOp = (SelectOperator) joinEnum.findASelectOp(leafInput);
-        if (selOp == null) { // add a SelectOperator with TRUE condition. The code below becomes simpler with a select operator.
+
+        if (selOp == null) { // this should not happen. So check later why this happening.
+            // add a SelectOperator with TRUE condition. The code below becomes simpler with a select operator.
             selOp = new SelectOperator(new MutableObject<>(ConstantExpression.TRUE));
             ILogicalOperator op = selOp;
             op.getInputs().add(new MutableObject<>(leafInput));
             leafInput = op;
         }
+
         ILogicalOperator parent = joinEnum.findDataSourceScanOperatorParent(leafInput);
         Mutable<ILogicalOperator> ref = new MutableObject<>(leafInput);
 
@@ -300,14 +346,16 @@
         parent.getInputs().get(0).setValue(deepCopyofScan);
         finalDatasetCard = origDatasetCard = idxDetails.getSourceCardinality();
         sampleCard = Math.min(idxDetails.getSampleCardinalityTarget(), origDatasetCard);
-        boolean unnest = joinEnum.findUnnestOp(selOp);
-        if (unnest) {
+        for (int i = 1; i <= numArrayRefs; i++) {
+            sampleCard = Math.min(idxDetails.getSampleCardinalityTarget(), origDatasetCard);
             ILogicalExpression saveExpr = selOp.getCondition().getValue();
-            double unnestSampleCard = joinEnum.stats.computeUnnestedOriginalCardinality(selOp);
+            double unnestSampleCard =
+                    joinEnum.stats.computeUnnestedOriginalCardinality(leafInput, leafInputNumber, numArrayRefs, i);
             selOp.getCondition().setValue(saveExpr); // restore the expression
             unnestFactor = unnestSampleCard / sampleCard;
-            sampleCard = unnestSampleCard;
-            finalDatasetCard = origDatasetCard = origDatasetCard * unnestFactor;
+            setCardsAndSizesForFakeJn(leafInputNumber, i, unnestFactor);
+            finalDatasetCard = origDatasetCard;
+            removeUnnestOp(leafInput); // remove the unnest op that was added in computeUnnestedOriginalCardinality
         }
         if (sampleCard == 0) { // should not happen unless the original dataset is empty
             sampleCard = 1; // we may have to make some adjustments to costs when the sample returns very rows.
@@ -373,6 +421,15 @@
         setAvgDocSize(idxDetails.getSourceAvgItemSize());
     }
 
+    private void removeUnnestOp(ILogicalOperator op) { // There will be only one UnnestOp for now at the top, so a while is strictly not necessary
+        ILogicalOperator parent = op;
+        op = op.getInputs().get(0).getValue(); // skip the select on the top
+        while (op.getOperatorTag() == LogicalOperatorTag.UNNEST) {
+            op = op.getInputs().get(0).getValue();
+        }
+        parent.getInputs().get(0).setValue(op);
+    }
+
     /** one is a subset of two */
     private boolean subset(int one, int two) {
         return (one & two) == one;
@@ -446,7 +503,10 @@
         if (this.applicableJoinConditions.size() >= 3) {
             redundantSel = removeRedundantPred(this.applicableJoinConditions);
         }
-
+        // mark all conditions back to non deleted state
+        for (JoinCondition jc : joinConditions) {
+            jc.deleted = false;
+        }
         // By dividing by redundantSel, we are undoing the earlier multiplication of all the selectivities.
         return joinCard / redundantSel;
     }
@@ -456,7 +516,7 @@
         if (jc1.comparisonType == JoinCondition.comparisonOp.OP_EQ
                 && jc2.comparisonType == JoinCondition.comparisonOp.OP_EQ
                 && jc3.comparisonType == JoinCondition.comparisonOp.OP_EQ) {
-            sel = findRedundantSel(jc1.selectivity, jc2.selectivity, jc3.selectivity);
+            sel = findRedundantSel(jc1, jc2, jc3);
         } else {
             // at least one of the predicates in not an equality predicate
             //this can get messy here, as 1, or 2 or all 3 can be non equality
@@ -472,6 +532,35 @@
         return sel;
     }
 
+    private static double findRedundantSel(JoinCondition jc1, JoinCondition jc2, JoinCondition jc3) {
+        // find middle selectivity
+        if (jc2.selectivity <= jc1.selectivity && jc1.selectivity <= jc3.selectivity) {
+            jc1.deleted = true;
+            return jc1.selectivity;
+        }
+        if (jc3.selectivity <= jc1.selectivity && jc1.selectivity <= jc2.selectivity) {
+            jc1.deleted = true;
+            return jc1.selectivity;
+        }
+        if (jc1.selectivity <= jc2.selectivity && jc2.selectivity <= jc3.selectivity) {
+            jc2.deleted = true;
+            return jc2.selectivity;
+        }
+        if (jc3.selectivity <= jc2.selectivity && jc2.selectivity <= jc1.selectivity) {
+            jc2.deleted = true;
+            return jc2.selectivity;
+        }
+        if (jc1.selectivity <= jc3.selectivity && jc3.selectivity <= jc2.selectivity) {
+            jc3.deleted = true;
+            return jc3.selectivity;
+        }
+        if (jc2.selectivity <= jc3.selectivity && jc3.selectivity <= jc1.selectivity) {
+            jc3.deleted = true;
+            return jc3.selectivity;
+        }
+        return 1.0; // keep compiler happy
+    }
+
     // if a redundant edge is found, we need to eliminate one of the edges.
     // If two triangles share an edge, removing the common edge will suffice
     // Each edge has two vertices. So we can only handle predicate with exactly two tables such as R.a = S.a
@@ -485,21 +574,21 @@
         int[] verticesCopy = new int[6];
         for (int i = 0; i <= applicablePredicatesInCurrentJn.size() - 3; i++) {
             jc1 = joinConditions.get(applicablePredicatesInCurrentJn.get(i));
-            if (jc1.partOfComposite) {
+            if (jc1.partOfComposite || jc1.deleted) {
                 continue; // must ignore these or the same triangles will be found more than once.
             }
             vertices[0] = jc1.leftSideBits;
             vertices[1] = jc1.rightSideBits;
             for (int j = i + 1; j <= applicablePredicatesInCurrentJn.size() - 2; j++) {
                 jc2 = joinConditions.get(applicablePredicatesInCurrentJn.get(j));
-                if (jc2.partOfComposite) {
+                if (jc2.partOfComposite || jc2.deleted) {
                     continue;
                 }
                 vertices[2] = jc2.leftSideBits;
                 vertices[3] = jc2.rightSideBits;
                 for (int k = j + 1; k <= applicablePredicatesInCurrentJn.size() - 1; k++) {
                     jc3 = joinConditions.get(applicablePredicatesInCurrentJn.get(k));
-                    if (jc3.partOfComposite) {
+                    if (jc3.partOfComposite || jc3.deleted) {
                         continue;
                     }
                     vertices[4] = jc3.leftSideBits;
@@ -510,7 +599,9 @@
                     if (verticesCopy[0] == verticesCopy[1] && verticesCopy[2] == verticesCopy[3]
                             && verticesCopy[4] == verticesCopy[5]) {
                         // redundant edge found
-                        redundantSel *= adjustSelectivities(jc1, jc2, jc3);
+                        if (!(jc1.deleted || jc2.deleted || jc3.deleted)) {
+                            redundantSel *= adjustSelectivities(jc1, jc2, jc3);
+                        }
                     }
                 }
             }
@@ -518,16 +609,6 @@
         return redundantSel;
     }
 
-    private static double findRedundantSel(double sel1, double sel2, double sel3) {
-        double[] sels = new double[3];
-        sels[0] = sel1;
-        sels[1] = sel2;
-        sels[2] = sel3;
-
-        Arrays.sort(sels); // we are sorting to make this deterministic
-        return sels[1]; // the middle one is closest to one of the extremes
-    }
-
     protected int addSingleDatasetPlans() {
         List<PlanNode> allPlans = joinEnum.allPlans;
         ICost opCost;
@@ -931,13 +1012,13 @@
             return false; // This should not happen. So debug to find out why this happened.
         }
 
-        if (innerLeafInput == joinLeafInput0) {
-            joinEnum.localJoinOp.getInputs().get(0).setValue(joinLeafInput1);
+        if (innerLeafInput == joinLeafInput0) { // skip the Select Operator with condition(TRUE) on top
+            joinEnum.localJoinOp.getInputs().get(0).setValue(joinLeafInput1.getInputs().get(0).getValue());
         } else {
-            joinEnum.localJoinOp.getInputs().get(0).setValue(joinLeafInput0);
+            joinEnum.localJoinOp.getInputs().get(0).setValue(joinLeafInput0.getInputs().get(0).getValue());
         }
 
-        joinEnum.localJoinOp.getInputs().get(1).setValue(innerLeafInput);
+        joinEnum.localJoinOp.getInputs().get(1).setValue(innerLeafInput.getInputs().get(0).getValue());
 
         // We will always use the first join Op to provide the joinOp input for invoking rewritePre
         AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) joinEnum.localJoinOp;
@@ -954,6 +1035,10 @@
     private boolean NLJoinApplicable(JoinNode leftJn, JoinNode rightJn, boolean outerJoin,
             ILogicalExpression nestedLoopJoinExpr, List<Pair<IAccessMethod, Index>> chosenIndexes,
             Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) throws AlgebricksException {
+
+        if (leftJn.fake || rightJn.fake) {
+            return false;
+        }
         if (nullExtendingSide(leftJn.datasetBits, outerJoin)) {
             return false;
         }
@@ -972,6 +1057,11 @@
     }
 
     private boolean CPJoinApplicable(JoinNode leftJn, boolean outerJoin) {
+
+        if (leftJn.fake || rightJn.fake) {
+            return false;
+        }
+
         if (!joinEnum.cboCPEnumMode) {
             return false;
         }
@@ -1033,6 +1123,10 @@
         this.leftJn = leftJn;
         this.rightJn = rightJn;
 
+        //if (leftJn.fake || rightJn.fake) { // uncomment if broadcast hash joins are not applicable
+        //return PlanNode.NO_PLAN;
+        //}
+
         if (!hashJoinApplicable(leftJn, outerJoin, hashJoinExpr)) {
             return PlanNode.NO_PLAN;
         }
@@ -1076,6 +1170,10 @@
         this.leftJn = leftJn;
         this.rightJn = rightJn;
 
+        if (leftJn.fake || rightJn.fake) {
+            return PlanNode.NO_PLAN;
+        }
+
         List<Pair<IAccessMethod, Index>> chosenIndexes = new ArrayList<>();
         Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs = new TreeMap<>();
         if (!NLJoinApplicable(leftJn, rightJn, outerJoin, nestedLoopJoinExpr, chosenIndexes, analyzedAMs)) {
@@ -1133,6 +1231,10 @@
         JoinNode leftJn = leftPlan.getJoinNode();
         JoinNode rightJn = rightPlan.getJoinNode();
 
+        if (leftJn.fake || rightJn.fake) {
+            return PlanNode.NO_PLAN;
+        }
+
         // Now build a cartesian product nested loops plan
         List<PlanNode> allPlans = joinEnum.allPlans;
         PlanNode pn;
@@ -1194,6 +1296,7 @@
             }
             leftPlan = joinEnum.allPlans.get(leftJn.cheapestPlanIndex);
             rightPlan = joinEnum.allPlans.get(rightJn.cheapestPlanIndex);
+
             addMultiDatasetPlans(leftPlan, rightPlan);
         } else {
             // FOR JOIN NODE LEVELS LESS THAN OR EQUAL TO THE LEVEL SPECIFIED FOR FULL ENUMERATION,
@@ -1492,6 +1595,7 @@
         StringBuilder sb = new StringBuilder(128);
         if (IsBaseLevelJoinNode()) {
             sb.append("Printing Scan Node ");
+            sb.append("Fake " + getFake() + " ");
         } else {
             sb.append("Printing Join Node ");
         }
@@ -1605,7 +1709,7 @@
                 lowestCostPlanIndex = planIndex;
             }
         }
-        sb.append("Cheapest Plan is ").append(lowestCostPlanIndex).append(", Cost is ")
+        sb.append("END Cheapest Plan is ").append(lowestCostPlanIndex).append(", Cost is ")
                 .append(dumpDouble(minCost.computeTotalCost()));
     }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/Stats.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/Stats.java
index aa2f40f..1cc643a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/Stats.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/Stats.java
@@ -61,9 +61,11 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
@@ -118,9 +120,7 @@
             // Since there is a left and right dataset here, expecting only two variables.
             return 1.0;
         }
-        if (!(joinExpr.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.EQ))) {
-            return 0.5; // we will assume half; rest of the code assumes EQ joins
-        }
+
         int idx1, idx2;
         if (joinEnum.varLeafInputIds.containsKey(exprUsedVars.get(0))) {
             idx1 = joinEnum.varLeafInputIds.get(exprUsedVars.get(0));
@@ -131,6 +131,12 @@
         } else
             return 1.0;
 
+        if (joinEnum.jnArray[idx1].getFake()) {
+            return 1.0;
+        }
+        if (joinEnum.jnArray[idx2].getFake()) {
+            return 1.0;
+        }
         double card1 = joinEnum.getJnArray()[idx1].origCardinality;
         double card2 = joinEnum.getJnArray()[idx2].origCardinality;
         if (card1 == 0.0 || card2 == 0.0) // should not happen
@@ -168,6 +174,9 @@
         } else {
             ILogicalOperator leafInput;
             LogicalVariable var;
+            if (!(joinExpr.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.EQ))) {
+                return 0.5; // we will assume half; rest of the code assumes EQ joins
+            }
             // choose the smaller side sample; better results this way for sure!
             if (card1 < card2) {
                 leafInput = joinEnum.leafInputs.get(idx1 - 1);
@@ -185,9 +194,29 @@
                 return 1.0;
             }
 
-            double estDistinctCardinalityFromSample = findPredicateCardinality(result, false);
-            double numDistincts = distinctEstimator2(estDistinctCardinalityFromSample, index);
-            return 1.0 / numDistincts; // this is the expected selectivity for joins.
+            double estDistinctCardinalityFromSample = findPredicateCardinality(result, true);
+            if (estDistinctCardinalityFromSample == 0) {
+                estDistinctCardinalityFromSample = 1; // just in case
+            }
+            Index.SampleIndexDetails details = (Index.SampleIndexDetails) index.getIndexDetails();
+            double numDistincts;
+            // if the table is smaller than the sample size, there is no need to use the estimator
+            //                                            getSampleCardinalityTarget() equals 1063 or 4252 or 17008
+            if (details.getSourceCardinality() <= details.getSampleCardinalityTarget()) {
+                numDistincts = estDistinctCardinalityFromSample;
+            } else { // when the number of distincts is smaller than approx 25% of the sample size, then we do not
+                         // then we do not need to call the estimator. This is a good heuristic. This was obtained by looking at the graph
+                     // of d = D ( 1 - e^(-getSampleCardinalityTarget/D) ; d = estDistinctCardinalityFromSample; D = actual number of distincts
+                if (estDistinctCardinalityFromSample <= 0.25 * details.getSampleCardinalityTarget()) {
+                    numDistincts = estDistinctCardinalityFromSample;
+                } else {
+                    numDistincts = secondDistinctEstimator(estDistinctCardinalityFromSample, index);
+                }
+            }
+            if (numDistincts > details.getSourceCardinality()) {
+                numDistincts = details.getSourceCardinality(); // cannot exceed table cardinality
+            }
+            return 1.0 / numDistincts; // this is the expected selectivity for joins for Fk-PK and Fk-Fk joins
         }
     }
 
@@ -415,6 +444,17 @@
         }
     }
 
+    protected void issueWarning(double sampleCard, DataSourceScanOperator scanOp) {
+        if (sampleCard == 0) {
+            sampleCard = 1;
+            IWarningCollector warningCollector = optCtx.getWarningCollector();
+            if (warningCollector.shouldWarn()) {
+                warningCollector.warn(Warning.of(scanOp.getSourceLocation(),
+                        org.apache.asterix.common.exceptions.ErrorCode.SAMPLE_HAS_ZERO_ROWS));
+            }
+        }
+    }
+
     protected double findSelectivityForThisPredicate(SelectOperator selOp, AbstractFunctionCallExpression exp,
             boolean arrayIndex) throws AlgebricksException {
         // replace the SelOp.condition with the new exp and replace it at the end
@@ -435,14 +475,7 @@
         Index.SampleIndexDetails idxDetails = (Index.SampleIndexDetails) index.getIndexDetails();
         double origDatasetCard = idxDetails.getSourceCardinality();
         double sampleCard = Math.min(idxDetails.getSampleCardinalityTarget(), origDatasetCard);
-        if (sampleCard == 0) {
-            sampleCard = 1;
-            IWarningCollector warningCollector = optCtx.getWarningCollector();
-            if (warningCollector.shouldWarn()) {
-                warningCollector.warn(Warning.of(scanOp.getSourceLocation(),
-                        org.apache.asterix.common.exceptions.ErrorCode.SAMPLE_HAS_ZERO_ROWS));
-            }
-        }
+        issueWarning(sampleCard, scanOp);
 
         // replace the dataScanSourceOperator with the sampling source
         SampleDataSource sampledatasource = joinEnum.getSampleDataSource(scanOp);
@@ -512,12 +545,18 @@
             // SELECT count(*) as revenue
             // FROM   orders o, o.o_orderline ol
             // WHERE  TRUE;
-            sampleCard = computeUnnestedOriginalCardinality(selOp);
+            sampleCard = computeUnnestedOriginalCardinality(selOp, 0, 0, 0);
         }
         // switch  the scanOp back
         parent.getInputs().get(0).setValue(scanOp);
 
-        double sel = (double) predicateCardinality / sampleCard;
+        double sel;
+
+        if (sampleCard >= 1.0) {
+            sel = (double) predicateCardinality / sampleCard;
+        } else {
+            sel = 0.0;
+        }
         return sel;
     }
 
@@ -537,12 +576,24 @@
         return record.numberOfFields();
     }
 
-    public double computeUnnestedOriginalCardinality(SelectOperator selOp) throws AlgebricksException {
+    public double computeUnnestedOriginalCardinality(ILogicalOperator leafInput, int leafInputNumber, int numArrayRefs,
+            int arrayRef) throws AlgebricksException {
         // Replace ALL SELECTS with TRUE, restore them after running the sampling query.
-        List<ILogicalExpression> selExprs = storeSelectConditionsAndMakeThemTrue(selOp, null);
-        List<List<IAObject>> result = runSamplingQuery(optCtx, selOp);
-        restoreAllSelectConditions(selOp, selExprs, null);
+        // Add the corresponding UnnestOp just below the top; these will be removed later.
+        if (leafInputNumber > 0) {
+            ILogicalOperator saveInput = leafInput.getInputs().get(0).getValue();
+            int size = joinEnum.unnestOpsInfo.get(leafInputNumber - 1).get(arrayRef - 1).size();
+            ILogicalOperator unnestOp = joinEnum.unnestOpsInfo.get(leafInputNumber - 1).get(arrayRef - 1).get(size - 1);
+            leafInput.getInputs().get(0).setValue(unnestOp);
+            unnestOp.getInputs().get(0).setValue(saveInput);
+        }
+
+        // Add the corresponding UnnestOperator just below the top.
+        List<ILogicalExpression> selExprs = storeSelectConditionsAndMakeThemTrue(leafInput, null);
+        List<List<IAObject>> result = runSamplingQuery(optCtx, leafInput);
+        restoreAllSelectConditions(leafInput, selExprs, null);
         return findPredicateCardinality(result, false);
+
     }
 
     public double findSizeVarsFromDisk(List<List<IAObject>> result, int numDiskVars) {
@@ -629,6 +680,49 @@
         return index;
     }
 
+    // creates assign [$$79] <- [{"$1": $$73, "$2": $$74, "$3": $$75, "$4": $$76, "$5": $$77, "$6": $$78}] and calls sampling query
+    protected List<List<IAObject>> helperFunction(IOptimizationContext newCtx, AggregateOperator newAggOp)
+            throws AlgebricksException {
+
+        Mutable<ILogicalOperator> newAggOpRef = new MutableObject<>(newAggOp);
+        OperatorPropertiesUtil.typeOpRec(newAggOpRef, newCtx); // is this really needed??
+
+        List<MutableObject> arr = createMutableObjectArray(newAggOp.getVariables());
+        AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
+                FunctionUtil.getFunctionInfo(BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR));
+        for (int i = 0; i < arr.size(); i++) {
+            f.getArguments().add(arr.get(i));
+        }
+
+        LogicalVariable newVar = newCtx.newVar();
+        AssignOperator assignOp = new AssignOperator(newVar, new MutableObject<>(f));
+        assignOp.getInputs().add(new MutableObject<>(newAggOp));
+        ProjectOperator pOp = new ProjectOperator(newVar);
+        pOp.getInputs().add(new MutableObject<>(assignOp));
+
+        Mutable<ILogicalOperator> newpOpRef = new MutableObject<>(pOp);
+
+        OperatorPropertiesUtil.typeOpRec(newpOpRef, newCtx);
+
+        if (LOGGER.isTraceEnabled()) {
+            String viewInPlan = new ALogicalPlanImpl(newpOpRef).toString(); //useful when debugging
+            LOGGER.trace("viewInPlan");
+            LOGGER.trace(viewInPlan);
+        }
+        LOGGER.info("*** calling sample query***");
+        return AnalysisUtil.runQuery(newpOpRef, Arrays.asList(newVar), newCtx, IRuleSetFactory.RuleSetKind.SAMPLING);
+    }
+
+    // plan we need to generate in this routine.
+    //  project ([$$36])                                 add here
+    //    assign [$$36] <- [{"$1": $$39}]                add here
+    //      aggregate [$$39] <- [agg-sql-count($$34)]    add here
+    //        distinct ([$$34])                          add here
+    //          project ([$$34])                         add here
+    //            assign [$$34] <- [$$ar.getField("country")]      part of leaf input
+    //              data-scan []<-[$$37, $$ar, $$38] <- `travel-sample`.inventory.airport
+    //                empty-tuple-source
+
     protected List<List<IAObject>> runSamplingQueryDistinct(IOptimizationContext ctx, ILogicalOperator logOp,
             LogicalVariable var, Index index) throws AlgebricksException {
         LOGGER.info("***running sample query***");
@@ -636,9 +730,10 @@
         IOptimizationContext newCtx = ctx.getOptimizationContextFactory().cloneOptimizationContext(ctx);
 
         ILogicalOperator newLogOp = OperatorManipulationUtil.bottomUpCopyOperators(logOp);
-        storeSelectConditionsAndMakeThemTrue(newLogOp, null);
+
         // by passing in null, all select expression will become true.
         // no need to restore them either as this is dne on a copy of the logOp.
+        storeSelectConditionsAndMakeThemTrue(newLogOp, null);
 
         ILogicalOperator parent = joinEnum.findDataSourceScanOperatorParent(newLogOp);
         DataSourceScanOperator scanOp;
@@ -650,14 +745,7 @@
         Index.SampleIndexDetails idxDetails = (Index.SampleIndexDetails) index.getIndexDetails();
         double origDatasetCard = idxDetails.getSourceCardinality();
         double sampleCard = Math.min(idxDetails.getSampleCardinalityTarget(), origDatasetCard);
-        if (sampleCard == 0) {
-            sampleCard = 1;
-            IWarningCollector warningCollector = optCtx.getWarningCollector();
-            if (warningCollector.shouldWarn()) {
-                warningCollector.warn(Warning.of(scanOp.getSourceLocation(),
-                        org.apache.asterix.common.exceptions.ErrorCode.SAMPLE_HAS_ZERO_ROWS));
-            }
-        }
+        issueWarning(sampleCard, scanOp);
 
         // replace the dataScanSourceOperator with the sampling source
         SampleDataSource sampledatasource = joinEnum.getSampleDataSource(scanOp);
@@ -671,14 +759,25 @@
             scanOp.setDataSource(sampledatasource);
         }
 
-        List<Mutable<ILogicalExpression>> aggFunArgs = new ArrayList<>(1);
-        aggFunArgs.add(new MutableObject<>(ConstantExpression.TRUE));
-
         AbstractLogicalExpression inputVarRef = new VariableReferenceExpression(var, newLogOp.getSourceLocation());
+        // add a project operator on top of newLogOp
+        ProjectOperator projOp = new ProjectOperator(var);
+        projOp.getInputs().add(new MutableObject<>(null)); //add an input
+        projOp.getInputs().get(0).setValue(newLogOp);
+        // add a distinct operator on top of the proj.
+        List<Mutable<ILogicalExpression>> arguments = new ArrayList<>();
+        VariableReferenceExpression e1 = new VariableReferenceExpression(var);
+        arguments.add(new MutableObject<>(e1));
+        DistinctOperator distOp = new DistinctOperator(arguments);
+        distOp.getInputs().add(new MutableObject<>(null)); //add an input
+        distOp.getInputs().get(0).setValue(projOp);
+        distOp.setExecutionMode(AbstractLogicalOperator.ExecutionMode.PARTITIONED);
+
+        // now add aggregate [$$39] <- [agg-sql-count($$34)] on top of distop
         List<Mutable<ILogicalExpression>> fields = new ArrayList<>(1);
         fields.add(new MutableObject<>(inputVarRef));
 
-        BuiltinFunctionInfo countFn = BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.SQL_COUNT_DISTINCT);
+        BuiltinFunctionInfo countFn = BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.SQL_COUNT);
         AggregateFunctionCallExpression aggExpr = new AggregateFunctionCallExpression(countFn, false, fields);
 
         List<Mutable<ILogicalExpression>> aggExprList = new ArrayList<>(1);
@@ -689,17 +788,10 @@
         aggVarList.add(aggVar);
 
         AggregateOperator newAggOp = new AggregateOperator(aggVarList, aggExprList);
-        newAggOp.getInputs().add(new MutableObject<>(newLogOp));
+        newAggOp.getInputs().add(new MutableObject<>(distOp));
 
-        Mutable<ILogicalOperator> newAggOpRef = new MutableObject<>(newAggOp);
-
-        OperatorPropertiesUtil.typeOpRec(newAggOpRef, newCtx);
-        LOGGER.info("***returning from sample query***");
-
-        String viewInPlan = new ALogicalPlanImpl(newAggOpRef).toString(); //useful when debugging
-        LOGGER.trace("viewInPlan");
-        LOGGER.trace(viewInPlan);
-        return AnalysisUtil.runQuery(newAggOpRef, Arrays.asList(aggVar), newCtx, IRuleSetFactory.RuleSetKind.SAMPLING);
+        // now add assign [$$36] <- [{"$1": $$39}]   on top of newAggOp; use the HelperFunction and call Sampling query
+        return helperFunction(newCtx, newAggOp);
     }
 
     // This one gets the cardinality and also projection sizes
@@ -788,32 +880,7 @@
         // add assign [$$79] <- [{"$1": $$73, "$2": $$74, "$3": $$75, "$4": $$76, "$5": $$77, "$6": $$78}]
         AggregateOperator newAggOp = new AggregateOperator(newVars2, aggExprList);
         newAggOp.getInputs().add(new MutableObject<>(assignOp));
-        Mutable<ILogicalOperator> newAggOpRef = new MutableObject<>(newAggOp);
-        OperatorPropertiesUtil.typeOpRec(newAggOpRef, newCtx); // is this really needed??
-        List<MutableObject> arr = createMutableObjectArray(newAggOp.getVariables());
-        AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
-                FunctionUtil.getFunctionInfo(BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR));
-        for (int i = 0; i < arr.size(); i++) {
-            f.getArguments().add(arr.get(i));
-        }
-
-        newVar = newCtx.newVar();
-        assignOp = new AssignOperator(newVar, new MutableObject<>(f));
-        assignOp.getInputs().add(new MutableObject<>(newAggOp));
-        ProjectOperator pOp = new ProjectOperator(newVar);
-        pOp.getInputs().add(new MutableObject<>(assignOp));
-
-        Mutable<ILogicalOperator> Ref = new MutableObject<>(pOp);
-
-        OperatorPropertiesUtil.typeOpRec(Ref, newCtx);
-        if (LOGGER.isTraceEnabled()) {
-            String viewInPlan = new ALogicalPlanImpl(Ref).toString(); //useful when debugging
-            LOGGER.trace("sampling query before calling runQuery");
-            LOGGER.trace(viewInPlan);
-        }
-
-        LOGGER.info("***returning from projection sample query***");
-        return AnalysisUtil.runQuery(Ref, Arrays.asList(newVar), newCtx, IRuleSetFactory.RuleSetKind.SAMPLING);
+        return helperFunction(newCtx, newAggOp);
     }
 
     private List<MutableObject> createMutableObjectArray(List<LogicalVariable> vars) {
@@ -915,7 +982,10 @@
         return x;
     }
 
-    private double distinctEstimator2(double estDistinctCardinalityFromSample, Index index) throws AlgebricksException {
+    // This estimator use the fact that the equation d = D (1 - e^n/D) is a 1-1 functions. So given d, it can find D using a
+    // binary search, thus avoiding the Newton Raphson iteration which is more complex.
+    private double secondDistinctEstimator(double estDistinctCardinalityFromSample, Index index)
+            throws AlgebricksException {
 
         Index.SampleIndexDetails idxDetails = (Index.SampleIndexDetails) index.getIndexDetails();
         double origDatasetCardinality = idxDetails.getSourceCardinality();
@@ -936,6 +1006,9 @@
             else
                 Dmax = D - 1;
         }
+        if (D == 0.0) { // just in case!
+            D = 1.0;
+        }
         return D;
     }
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
index 01b8527..a39bc06 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
@@ -33,6 +33,7 @@
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Datatype;
 import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.om.types.ARecordType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 
@@ -605,6 +606,8 @@
         private final List<Expression> keyExpressions;
         private final boolean autogenerated;
 
+        private final ARecordType itemType;
+
         public CompiledCopyToStatement(CopyToStatement copyToStatement) {
             this.query = copyToStatement.getQuery();
             this.sourceVariable = copyToStatement.getSourceVariable();
@@ -619,6 +622,7 @@
             this.orderByNullModifierList = copyToStatement.getOrderByNullModifierList();
             this.keyExpressions = copyToStatement.getKeyExpressions();
             this.autogenerated = copyToStatement.isAutogenerated();
+            this.itemType = eddDecl.getItemType();
         }
 
         @Override
@@ -642,6 +646,10 @@
             return properties;
         }
 
+        public ARecordType getItemType() {
+            return itemType;
+        }
+
         public List<Expression> getPathExpressions() {
             return pathExpressions;
         }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index d5c916f..978997c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -466,7 +466,8 @@
         }
 
         // Write adapter configuration
-        WriteDataSink writeDataSink = new WriteDataSink(copyTo.getAdapter(), copyTo.getProperties());
+        WriteDataSink writeDataSink = new WriteDataSink(copyTo.getAdapter(), copyTo.getProperties(),
+                copyTo.getItemType(), expr.getSourceLocation());
 
         // writeOperator
         WriteOperator writeOperator = new WriteOperator(sourceExprRef, new MutableObject<>(fullPathExpr),
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
index 302d4e0..3d806a8 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
@@ -157,7 +157,7 @@
 
     public static final String REWRITE_IN_AS_OR_OPTION = "rewrite_in_as_or";
     private static final boolean REWRITE_IN_AS_OR_OPTION_DEFAULT = true;
-
+    public static final String ARRAY_ACCESS = "array_access";
     private final Map<VarIdentifier, IAObject> externalVars;
     private final boolean translateInAsOr;
 
@@ -330,6 +330,7 @@
         } else {
             unnestOp = new UnnestOperator(fromVar, new MutableObject<>(pUnnestExpr.first));
         }
+        unnestOp.getAnnotations().put(ARRAY_ACCESS, fromExpr.getKind() == Kind.FIELD_ACCESSOR_EXPRESSION);
         ExternalSubpathAnnotation hint = ((AbstractExpression) fromExpr).findHint(ExternalSubpathAnnotation.class);
         if (hint != null) {
             unnestOp.getAnnotations().put(SUBPATH, hint.getSubPath());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
index 34976b6..579ad51 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
@@ -276,6 +276,7 @@
                     case UUID:
                     case YEARMONTHDURATION:
                     case DAYTIMEDURATION:
+                    case ANY:
                         break;
                     default:
                         throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
diff --git a/asterixdb/asterix-app/data/heterogeneousData.adm b/asterixdb/asterix-app/data/heterogeneousData.adm
new file mode 100644
index 0000000..0d96209
--- /dev/null
+++ b/asterixdb/asterix-app/data/heterogeneousData.adm
@@ -0,0 +1,8 @@
+{ "userId": 1, "email": "john.doe@example.com", "username": "johndoe", "isActive": true, "occupation": "teacher","age": 23}
+{ "userId": 2, "email": "jane.smith@example.com", "username": "janesmith", "isActive": false, "occupation": "firefighter" }
+{ "userId": 3, "email": "michael.brown@example.com", "username": "michaelb", "isActive": true, "age": "five", "occupation":4 }
+{ "userId": 4, "email": "emily.davis@example.com", "username": "emilyd", "isActive": true, "age": 23.1 }
+{ "userId": 5, "email": "chris.jones@example.com", "username": "chrisj", "isActive": false, "age": "old" }
+{ "userId": 6, "email": "patricia.garcia@example.com", "username": "patriciag", "isActive": true, "age": "45" }
+{ "userId": 7, "email": "linda.martinez@example.com", "username": "lindam", "isActive": false }
+{ "userId": 8, "email": "robert.lee@example.com", "username": "robertl", "isActive": true, "age": 31 }
diff --git a/asterixdb/asterix-app/pom.xml b/asterixdb/asterix-app/pom.xml
index 1b6ec5e..e9d7412 100644
--- a/asterixdb/asterix-app/pom.xml
+++ b/asterixdb/asterix-app/pom.xml
@@ -422,36 +422,6 @@
           </execution>
         </executions>
       </plugin>
-      <plugin>
-        <groupId>com.github.eirslett</groupId>
-        <artifactId>frontend-maven-plugin</artifactId>
-        <version>1.13.4</version>
-        <configuration>
-          <nodeVersion>v14.15.4</nodeVersion>
-          <npmVersion>6.14.11</npmVersion>
-          <workingDirectory>target/npm</workingDirectory>
-          <installDirectory>target/npm</installDirectory>
-        </configuration>
-        <executions>
-          <execution>
-            <id>install node and yarn</id>
-            <goals>
-              <goal>install-node-and-npm</goal>
-            </goals>
-            <phase>${azurite.npm.install.stage}</phase>
-          </execution>
-          <execution>
-            <id>azurite blob</id>
-            <phase>${azurite.install.stage}</phase>
-            <goals>
-              <goal>npm</goal>
-            </goals>
-            <configuration>
-              <arguments>install azurite@3.31.0</arguments>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
     </plugins>
     <pluginManagement>
       <plugins>
@@ -513,7 +483,7 @@
     <profile>
       <id>asterix-gerrit-asterix-app</id>
       <properties>
-        <test.excludes>**/CloudStorageTest.java,**/CloudStorageGCSTest.java,**/SqlppExecutionWithCancellationTest.java,**/DmlTest.java,**/RepeatedTest.java,**/SqlppExecutionTest.java,**/SqlppExecutionColumnTest.java,**/*StaticPartitioning*Test.java,**/*Ssl*Test.java,**/Podman*.java,**/*AnalyzedExecutionTest.java,**/SqlppProfiledExecutionTest.java,**/CloudPythonTest.java</test.excludes>
+        <test.excludes>**/CloudStorageTest.java,**/CloudStorageGCSTest.java,**/SqlppExecutionWithCancellationTest.java,**/DmlTest.java,**/RepeatedTest.java,**/SqlppExecutionTest.java,**/SqlppExecutionColumnTest.java,**/*StaticPartitioning*Test.java,**/*Ssl*Test.java,**/Podman*.java,**/*AnalyzedExecutionTest.java,**/SqlppProfiledExecutionTest.java,**/CloudPythonTest.java,**/CloudStorageAzTest.java</test.excludes>
         <itest.excludes>**/*.java</itest.excludes>
       </properties>
       <build>
@@ -590,6 +560,43 @@
         <failIfNoTests>false</failIfNoTests>
       </properties>
     </profile>
+    <profile>
+        <id>azurite-tests</id>
+        <build>
+            <plugins>
+                <plugin>
+                    <groupId>com.github.eirslett</groupId>
+                    <artifactId>frontend-maven-plugin</artifactId>
+                    <version>1.13.4</version>
+                    <configuration>
+                        <nodeVersion>v14.15.4</nodeVersion>
+                        <npmVersion>6.14.11</npmVersion>
+                        <workingDirectory>target/npm</workingDirectory>
+                        <installDirectory>target/npm</installDirectory>
+                    </configuration>
+                    <executions>
+                        <execution>
+                            <id>install node and yarn</id>
+                            <goals>
+                                <goal>install-node-and-npm</goal>
+                            </goals>
+                            <phase>${azurite.npm.install.stage}</phase>
+                        </execution>
+                        <execution>
+                            <id>azurite blob</id>
+                            <phase>${azurite.install.stage}</phase>
+                            <goals>
+                                <goal>npm</goal>
+                            </goals>
+                            <configuration>
+                                <arguments>install azurite@3.31.0</arguments>
+                            </configuration>
+                        </execution>
+                    </executions>
+                </plugin>
+            </plugins>
+        </build>
+    </profile>
   </profiles>
   <dependencies>
     <dependency>
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index 47498ea..c5fc395 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -114,6 +114,8 @@
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.api.job.resource.IClusterCapacity;
 import org.apache.hyracks.control.common.config.OptionTypes;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectWriter;
@@ -125,6 +127,8 @@
  */
 public class APIFramework {
 
+    private static final Logger LOGGER = LogManager.getLogger();
+
     private static final ObjectWriter OBJECT_WRITER = new ObjectMapper().writerWithDefaultPrettyPrinter();
 
     public static final String PREFIX_INTERNAL_PARAMETERS = "_internal";
@@ -192,16 +196,21 @@
     public Pair<IReturningStatement, Integer> reWriteQuery(LangRewritingContext langRewritingContext,
             IReturningStatement q, SessionOutput output, boolean allowNonStoredUdfCalls, boolean inlineUdfsAndViews,
             Collection<VarIdentifier> externalVars) throws CompilationException {
-        if (q == null) {
-            return null;
+        try {
+            if (q == null) {
+                return null;
+            }
+            SessionConfig conf = output.config();
+            if (!conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS) && conf.is(SessionConfig.OOB_EXPR_TREE)) {
+                generateExpressionTree(q);
+            }
+            IQueryRewriter rw = rewriterFactory.createQueryRewriter();
+            rw.rewrite(langRewritingContext, q, allowNonStoredUdfCalls, inlineUdfsAndViews, externalVars);
+            return new Pair<>(q, q.getVarCounter());
+        } catch (StackOverflowError error) {
+            LOGGER.info("Stack Overflow", error);
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, "internal error");
         }
-        SessionConfig conf = output.config();
-        if (!conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS) && conf.is(SessionConfig.OOB_EXPR_TREE)) {
-            generateExpressionTree(q);
-        }
-        IQueryRewriter rw = rewriterFactory.createQueryRewriter();
-        rw.rewrite(langRewritingContext, q, allowNonStoredUdfCalls, inlineUdfsAndViews, externalVars);
-        return new Pair<>(q, q.getVarCounter());
     }
 
     public JobSpecification compileQuery(IClusterInfoCollector clusterInfoCollector, MetadataProvider metadataProvider,
@@ -210,167 +219,183 @@
             IRequestParameters requestParameters, EnumSet<JobFlag> runtimeFlags)
             throws AlgebricksException, ACIDException {
 
-        // establish facts
-        final boolean isQuery = query != null;
-        final boolean isLoad = statement != null && statement.getKind() == Statement.Kind.LOAD;
-        final boolean isCopy = statement != null && statement.getKind() == Statement.Kind.COPY_FROM;
-        final SourceLocation sourceLoc =
-                query != null ? query.getSourceLocation() : statement != null ? statement.getSourceLocation() : null;
-        final boolean isExplainOnly = isQuery && query.isExplain();
+        try {
+            // establish facts
+            final boolean isQuery = query != null;
+            final boolean isLoad = statement != null && statement.getKind() == Statement.Kind.LOAD;
+            final boolean isCopy = statement != null && statement.getKind() == Statement.Kind.COPY_FROM;
+            final SourceLocation sourceLoc = query != null ? query.getSourceLocation()
+                    : statement != null ? statement.getSourceLocation() : null;
+            final boolean isExplainOnly = isQuery && query.isExplain();
 
-        SessionConfig conf = output.config();
-        if (isQuery && !conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS)
-                && conf.is(SessionConfig.OOB_REWRITTEN_EXPR_TREE)) {
-            generateRewrittenExpressionTree(query);
-        }
+            SessionConfig conf = output.config();
+            if (isQuery && !conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS)
+                    && conf.is(SessionConfig.OOB_REWRITTEN_EXPR_TREE)) {
+                generateRewrittenExpressionTree(query);
+            }
 
-        final TxnId txnId = metadataProvider.getTxnIdFactory().create();
-        metadataProvider.setTxnId(txnId);
-        ILangExpressionToPlanTranslator t =
-                translatorFactory.createExpressionToPlanTranslator(metadataProvider, varCounter, externalVars);
-        ResultMetadata resultMetadata = new ResultMetadata(output.config().fmt());
-        ILogicalPlan plan = isLoad || isCopy ? t.translateCopyOrLoad((ICompiledDmlStatement) statement)
-                : t.translate(query, outputDatasetName, statement, resultMetadata);
+            final TxnId txnId = metadataProvider.getTxnIdFactory().create();
+            metadataProvider.setTxnId(txnId);
+            ILangExpressionToPlanTranslator t =
+                    translatorFactory.createExpressionToPlanTranslator(metadataProvider, varCounter, externalVars);
+            ResultMetadata resultMetadata = new ResultMetadata(output.config().fmt());
+            ILogicalPlan plan = isLoad || isCopy ? t.translateCopyOrLoad((ICompiledDmlStatement) statement)
+                    : t.translate(query, outputDatasetName, statement, resultMetadata);
 
-        ICcApplicationContext ccAppContext = metadataProvider.getApplicationContext();
-        CompilerProperties compilerProperties = ccAppContext.getCompilerProperties();
-        Map<String, Object> querySpecificConfig = validateConfig(metadataProvider.getConfig(), sourceLoc);
-        final PhysicalOptimizationConfig physOptConf = OptimizationConfUtil.createPhysicalOptimizationConf(
-                compilerProperties, querySpecificConfig, configurableParameterNames, sourceLoc);
-        boolean cboMode = physOptConf.getCBOMode() || physOptConf.getCBOTestMode();
-        HeuristicCompilerFactoryBuilder builder =
-                new HeuristicCompilerFactoryBuilder(OptimizationContextFactory.INSTANCE);
-        builder.setPhysicalOptimizationConfig(physOptConf);
-        builder.setLogicalRewrites(() -> ruleSetFactory.getLogicalRewrites(ccAppContext));
-        builder.setLogicalRewritesByKind(kind -> ruleSetFactory.getLogicalRewrites(kind, ccAppContext));
-        builder.setPhysicalRewrites(() -> ruleSetFactory.getPhysicalRewrites(ccAppContext));
-        IDataFormat format = metadataProvider.getDataFormat();
-        ICompilerFactory compilerFactory = builder.create();
-        builder.setExpressionEvalSizeComputer(format.getExpressionEvalSizeComputer());
-        builder.setIMergeAggregationExpressionFactory(new MergeAggregationExpressionFactory());
-        builder.setPartialAggregationTypeComputer(new PartialAggregationTypeComputer());
-        builder.setExpressionTypeComputer(ExpressionTypeComputer.INSTANCE);
-        builder.setMissableTypeComputer(MissableTypeComputer.INSTANCE);
-        builder.setConflictingTypeResolver(ConflictingTypeResolver.INSTANCE);
-        builder.setWarningCollector(warningCollector);
-        builder.setMaxWarnings(conf.getMaxWarnings());
+            ICcApplicationContext ccAppContext = metadataProvider.getApplicationContext();
+            CompilerProperties compilerProperties = ccAppContext.getCompilerProperties();
+            Map<String, Object> config = metadataProvider.getConfig();
+            Map<String, Object> querySpecificConfig = validateConfig(config, sourceLoc);
+            final PhysicalOptimizationConfig physOptConf = OptimizationConfUtil.createPhysicalOptimizationConf(
+                    compilerProperties, querySpecificConfig, configurableParameterNames, sourceLoc);
+            if (!config.containsKey(CompilerProperties.COMPILER_ORDERED_FIELDS_KEY)) {
+                config.put(CompilerProperties.COMPILER_ORDERED_FIELDS_KEY,
+                        Boolean.toString(physOptConf.isOrderField()));
+            }
 
-        if ((isQuery || isLoad || isCopy) && !conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS)
-                && conf.is(SessionConfig.OOB_LOGICAL_PLAN)) {
-            generateLogicalPlan(plan, output.config().getPlanFormat(), cboMode);
-        }
+            boolean cboMode = physOptConf.getCBOMode() || physOptConf.getCBOTestMode();
+            HeuristicCompilerFactoryBuilder builder =
+                    new HeuristicCompilerFactoryBuilder(OptimizationContextFactory.INSTANCE);
+            builder.setPhysicalOptimizationConfig(physOptConf);
+            builder.setLogicalRewrites(() -> ruleSetFactory.getLogicalRewrites(ccAppContext));
+            builder.setLogicalRewritesByKind(kind -> ruleSetFactory.getLogicalRewrites(kind, ccAppContext));
+            builder.setPhysicalRewrites(() -> ruleSetFactory.getPhysicalRewrites(ccAppContext));
+            IDataFormat format = metadataProvider.getDataFormat();
+            ICompilerFactory compilerFactory = builder.create();
+            builder.setExpressionEvalSizeComputer(format.getExpressionEvalSizeComputer());
+            builder.setIMergeAggregationExpressionFactory(new MergeAggregationExpressionFactory());
+            builder.setPartialAggregationTypeComputer(new PartialAggregationTypeComputer());
+            builder.setExpressionTypeComputer(ExpressionTypeComputer.INSTANCE);
+            builder.setMissableTypeComputer(MissableTypeComputer.INSTANCE);
+            builder.setConflictingTypeResolver(ConflictingTypeResolver.INSTANCE);
+            builder.setWarningCollector(warningCollector);
+            builder.setMaxWarnings(conf.getMaxWarnings());
 
-        int parallelism = getParallelism((String) querySpecificConfig.get(CompilerProperties.COMPILER_PARALLELISM_KEY),
-                compilerProperties.getParallelism());
-        AlgebricksAbsolutePartitionConstraint computationLocations =
-                chooseLocations(clusterInfoCollector, parallelism, metadataProvider.getClusterLocations());
-        builder.setClusterLocations(computationLocations);
+            if ((isQuery || isLoad || isCopy) && !conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS)
+                    && conf.is(SessionConfig.OOB_LOGICAL_PLAN)) {
+                generateLogicalPlan(plan, output.config().getPlanFormat(), cboMode);
+            }
 
-        builder.setBinaryBooleanInspectorFactory(format.getBinaryBooleanInspectorFactory());
-        builder.setBinaryIntegerInspectorFactory(format.getBinaryIntegerInspectorFactory());
-        builder.setComparatorFactoryProvider(format.getBinaryComparatorFactoryProvider());
-        builder.setExpressionRuntimeProvider(
-                new ExpressionRuntimeProvider(new QueryLogicalExpressionJobGen(metadataProvider.getFunctionManager())));
-        builder.setHashFunctionFactoryProvider(format.getBinaryHashFunctionFactoryProvider());
-        builder.setHashFunctionFamilyProvider(format.getBinaryHashFunctionFamilyProvider());
-        builder.setMissingWriterFactory(format.getMissingWriterFactory());
-        builder.setNullWriterFactory(format.getNullWriterFactory());
-        builder.setUnnestingPositionWriterFactory(format.getUnnestingPositionWriterFactory());
-        builder.setPredicateEvaluatorFactoryProvider(format.getPredicateEvaluatorFactoryProvider());
-        builder.setPrinterProvider(getPrinterFactoryProvider(format, conf.fmt()));
-        builder.setWriterFactory(PrinterBasedWriterFactory.INSTANCE);
-        builder.setResultSerializerFactoryProvider(ResultSerializerFactoryProvider.INSTANCE);
-        builder.setSerializerDeserializerProvider(format.getSerdeProvider());
-        builder.setTypeTraitProvider(format.getTypeTraitProvider());
-        builder.setNormalizedKeyComputerFactoryProvider(format.getNormalizedKeyComputerFactoryProvider());
+            int parallelism =
+                    getParallelism((String) querySpecificConfig.get(CompilerProperties.COMPILER_PARALLELISM_KEY),
+                            compilerProperties.getParallelism());
+            AlgebricksAbsolutePartitionConstraint computationLocations =
+                    chooseLocations(clusterInfoCollector, parallelism, metadataProvider.getClusterLocations());
+            builder.setClusterLocations(computationLocations);
 
-        ICompiler compiler = compilerFactory.createCompiler(plan, metadataProvider, t.getVarCounter());
-        if (conf.isOptimize()) {
-            compiler.optimize();
-            if (conf.is(SessionConfig.OOB_OPTIMIZED_LOGICAL_PLAN) || isExplainOnly) {
-                if (conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS)) {
-                    // For Optimizer tests. Print physical operators in verbose mode.
-                    AlgebricksStringBuilderWriter buf = new AlgebricksStringBuilderWriter(PlanPrettyPrinter.INIT_SIZE);
-                    PlanPrettyPrinter.printPhysicalOps(plan, buf, 0, true);
-                    output.out().write(buf.toString());
-                } else {
-                    if (isQuery || isLoad || isCopy) {
-                        generateOptimizedLogicalPlan(plan, output.config().getPlanFormat(), cboMode);
+            builder.setBinaryBooleanInspectorFactory(format.getBinaryBooleanInspectorFactory());
+            builder.setBinaryIntegerInspectorFactory(format.getBinaryIntegerInspectorFactory());
+            builder.setComparatorFactoryProvider(format.getBinaryComparatorFactoryProvider());
+            builder.setExpressionRuntimeProvider(new ExpressionRuntimeProvider(
+                    new QueryLogicalExpressionJobGen(metadataProvider.getFunctionManager())));
+            builder.setHashFunctionFactoryProvider(format.getBinaryHashFunctionFactoryProvider());
+            builder.setHashFunctionFamilyProvider(format.getBinaryHashFunctionFamilyProvider());
+            builder.setMissingWriterFactory(format.getMissingWriterFactory());
+            builder.setNullWriterFactory(format.getNullWriterFactory());
+            builder.setUnnestingPositionWriterFactory(format.getUnnestingPositionWriterFactory());
+            builder.setPredicateEvaluatorFactoryProvider(format.getPredicateEvaluatorFactoryProvider());
+            builder.setPrinterProvider(getPrinterFactoryProvider(format, conf.fmt()));
+            builder.setWriterFactory(PrinterBasedWriterFactory.INSTANCE);
+            builder.setResultSerializerFactoryProvider(ResultSerializerFactoryProvider.INSTANCE);
+            builder.setSerializerDeserializerProvider(format.getSerdeProvider());
+            builder.setTypeTraitProvider(format.getTypeTraitProvider());
+            builder.setNormalizedKeyComputerFactoryProvider(format.getNormalizedKeyComputerFactoryProvider());
+
+            ICompiler compiler = compilerFactory.createCompiler(plan, metadataProvider, t.getVarCounter());
+            if (conf.isOptimize()) {
+                compiler.optimize();
+                if (conf.is(SessionConfig.OOB_OPTIMIZED_LOGICAL_PLAN) || isExplainOnly) {
+                    if (conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS)) {
+                        // For Optimizer tests. Print physical operators in verbose mode.
+                        AlgebricksStringBuilderWriter buf =
+                                new AlgebricksStringBuilderWriter(PlanPrettyPrinter.INIT_SIZE);
+                        PlanPrettyPrinter.printPhysicalOps(plan, buf, 0, true);
+                        output.out().write(buf.toString());
+                    } else {
+                        if (isQuery || isLoad || isCopy) {
+                            generateOptimizedLogicalPlan(plan, output.config().getPlanFormat(), cboMode);
+                        }
                     }
                 }
             }
-        }
 
-        if (conf.getClientType() == SessionConfig.ClientType.JDBC) {
-            executionPlans.setStatementCategory(Statement.Category.toString(getStatementCategory(query, statement)));
-            if (!conf.isExecuteQuery()) {
-                String stmtParams = ResultUtil.ParseOnlyResult.printStatementParameters(externalVars.keySet(), v -> v);
-                executionPlans.setStatementParameters(stmtParams);
+            if (conf.getClientType() == SessionConfig.ClientType.JDBC) {
+                executionPlans
+                        .setStatementCategory(Statement.Category.toString(getStatementCategory(query, statement)));
+                if (!conf.isExecuteQuery()) {
+                    String stmtParams =
+                            ResultUtil.ParseOnlyResult.printStatementParameters(externalVars.keySet(), v -> v);
+                    executionPlans.setStatementParameters(stmtParams);
+                }
+                if (isExplainOnly) {
+                    executionPlans.setExplainOnly(true);
+                } else if (isQuery) {
+                    executionPlans.setSignature(SignaturePrinter.generateFlatSignature(resultMetadata));
+                }
             }
+
+            boolean printSignature = isQuery && requestParameters != null && requestParameters.isPrintSignature();
+
+            if (printSignature && !isExplainOnly) { //explainOnly adds the signature later
+                printer.addResultPrinter(SignaturePrinter.newInstance(executionPlans));
+            }
+
+            if (!conf.isGenerateJobSpec()) {
+                if (isQuery || isLoad || isCopy) {
+                    generateOptimizedLogicalPlan(plan, output.config().getPlanFormat(), cboMode);
+                }
+                return null;
+            }
+
+            JobEventListenerFactory jobEventListenerFactory =
+                    new JobEventListenerFactory(txnId, metadataProvider.isWriteTransaction());
+            JobSpecification spec = compiler.createJob(ccAppContext, jobEventListenerFactory, runtimeFlags);
+
+            if (isQuery || isCopy) {
+                if (!compiler.skipJobCapacityAssignment()) {
+                    if (requestParameters == null || !requestParameters.isSkipAdmissionPolicy()) {
+                        // Sets a required capacity, only for read-only queries.
+                        // DDLs and DMLs are considered not that frequent.
+                        // limit the computation locations to the locations that will be used in the query
+                        final INodeJobTracker nodeJobTracker = ccAppContext.getNodeJobTracker();
+                        final AlgebricksAbsolutePartitionConstraint jobLocations =
+                                getJobLocations(spec, nodeJobTracker, computationLocations);
+                        final IClusterCapacity jobRequiredCapacity =
+                                ResourceUtils.getRequiredCapacity(plan, jobLocations, physOptConf, compilerProperties);
+                        addRuntimeMemoryOverhead(jobRequiredCapacity, compilerProperties);
+                        spec.setRequiredClusterCapacity(jobRequiredCapacity);
+                    }
+                }
+            }
+
+            if (conf.is(SessionConfig.OOB_OPTIMIZED_LOGICAL_PLAN) || isExplainOnly) {
+                if (isQuery || isLoad || isCopy) {
+                    generateOptimizedLogicalPlan(plan, spec.getLogical2PhysicalMap(), output.config().getPlanFormat(),
+                            cboMode);
+                    if (runtimeFlags.contains(JobFlag.PROFILE_RUNTIME)) {
+                        lastPlan = new PlanInfo(plan, spec.getLogical2PhysicalMap(), cboMode,
+                                output.config().getPlanFormat());
+                    }
+                }
+            }
+
             if (isExplainOnly) {
-                executionPlans.setExplainOnly(true);
-            } else if (isQuery) {
-                executionPlans.setSignature(SignaturePrinter.generateFlatSignature(resultMetadata));
-            }
-        }
-
-        boolean printSignature = isQuery && requestParameters != null && requestParameters.isPrintSignature();
-
-        if (printSignature && !isExplainOnly) { //explainOnly adds the signature later
-            printer.addResultPrinter(SignaturePrinter.newInstance(executionPlans));
-        }
-
-        if (!conf.isGenerateJobSpec()) {
-            if (isQuery || isLoad || isCopy) {
-                generateOptimizedLogicalPlan(plan, output.config().getPlanFormat(), cboMode);
-            }
-            return null;
-        }
-
-        JobEventListenerFactory jobEventListenerFactory =
-                new JobEventListenerFactory(txnId, metadataProvider.isWriteTransaction());
-        JobSpecification spec = compiler.createJob(ccAppContext, jobEventListenerFactory, runtimeFlags);
-
-        if (isQuery || isCopy) {
-            if (!compiler.skipJobCapacityAssignment()) {
-                if (requestParameters == null || !requestParameters.isSkipAdmissionPolicy()) {
-                    // Sets a required capacity, only for read-only queries.
-                    // DDLs and DMLs are considered not that frequent.
-                    // limit the computation locations to the locations that will be used in the query
-                    final INodeJobTracker nodeJobTracker = ccAppContext.getNodeJobTracker();
-                    final AlgebricksAbsolutePartitionConstraint jobLocations =
-                            getJobLocations(spec, nodeJobTracker, computationLocations);
-                    final IClusterCapacity jobRequiredCapacity =
-                            ResourceUtils.getRequiredCapacity(plan, jobLocations, physOptConf, compilerProperties);
-                    addRuntimeMemoryOverhead(jobRequiredCapacity, compilerProperties);
-                    spec.setRequiredClusterCapacity(jobRequiredCapacity);
+                printPlanAsResult(metadataProvider, output, printer, printSignature);
+                if (!conf.is(SessionConfig.OOB_OPTIMIZED_LOGICAL_PLAN)) {
+                    executionPlans.setOptimizedLogicalPlan(null);
                 }
+                return null;
             }
-        }
 
-        if (conf.is(SessionConfig.OOB_OPTIMIZED_LOGICAL_PLAN) || isExplainOnly) {
-            if (isQuery || isLoad || isCopy) {
-                generateOptimizedLogicalPlan(plan, spec.getLogical2PhysicalMap(), output.config().getPlanFormat(),
-                        cboMode);
-                if (runtimeFlags.contains(JobFlag.PROFILE_RUNTIME)) {
-                    lastPlan =
-                            new PlanInfo(plan, spec.getLogical2PhysicalMap(), cboMode, output.config().getPlanFormat());
-                }
+            if (isQuery && conf.is(SessionConfig.OOB_HYRACKS_JOB)) {
+                generateJob(spec);
             }
-        }
+            return spec;
 
-        if (isExplainOnly) {
-            printPlanAsResult(metadataProvider, output, printer, printSignature);
-            if (!conf.is(SessionConfig.OOB_OPTIMIZED_LOGICAL_PLAN)) {
-                executionPlans.setOptimizedLogicalPlan(null);
-            }
-            return null;
+        } catch (StackOverflowError error) {
+            LOGGER.info("Stack Overflow", error);
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, "internal error");
         }
-
-        if (isQuery && conf.is(SessionConfig.OOB_HYRACKS_JOB)) {
-            generateJob(spec);
-        }
-        return spec;
     }
 
     private void printPlanAsResult(MetadataProvider metadataProvider, SessionOutput output, IResponsePrinter printer,
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CcApplicationContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CcApplicationContext.java
index 6ea7aeb..e892d04 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CcApplicationContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CcApplicationContext.java
@@ -24,6 +24,8 @@
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.function.Supplier;
 
+import org.apache.asterix.app.external.ExternalCredentialsCache;
+import org.apache.asterix.app.external.ExternalCredentialsCacheUpdater;
 import org.apache.asterix.app.result.ResultReader;
 import org.apache.asterix.common.api.IConfigValidator;
 import org.apache.asterix.common.api.IConfigValidatorFactory;
@@ -55,6 +57,8 @@
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.dataflow.IDataPartitioningProvider;
 import org.apache.asterix.common.external.IAdapterFactoryService;
+import org.apache.asterix.common.external.IExternalCredentialsCache;
+import org.apache.asterix.common.external.IExternalCredentialsCacheUpdater;
 import org.apache.asterix.common.metadata.IMetadataBootstrap;
 import org.apache.asterix.common.metadata.IMetadataLockUtil;
 import org.apache.asterix.common.replication.INcLifecycleCoordinator;
@@ -127,6 +131,8 @@
     private final IOManager ioManager;
     private final INamespacePathResolver namespacePathResolver;
     private final INamespaceResolver namespaceResolver;
+    private final IExternalCredentialsCache externalCredentialsCache;
+    private final IExternalCredentialsCacheUpdater externalCredentialsCacheUpdater;
 
     public CcApplicationContext(ICCServiceContext ccServiceCtx, HyracksConnection hcc,
             Supplier<IMetadataBootstrap> metadataBootstrapSupplier, IGlobalRecoveryManager globalRecoveryManager,
@@ -177,6 +183,8 @@
         this.globalTxManager = globalTxManager;
         this.ioManager = ioManager;
         dataPartitioningProvider = DataPartitioningProvider.create(this);
+        externalCredentialsCache = new ExternalCredentialsCache();
+        externalCredentialsCacheUpdater = new ExternalCredentialsCacheUpdater(this);
     }
 
     @Override
@@ -415,4 +423,14 @@
     public IOManager getIoManager() {
         return ioManager;
     }
+
+    @Override
+    public IExternalCredentialsCache getExternalCredentialsCache() {
+        return externalCredentialsCache;
+    }
+
+    @Override
+    public IExternalCredentialsCacheUpdater getExternalCredentialsCacheUpdater() {
+        return externalCredentialsCacheUpdater;
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalCredentialsCache.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalCredentialsCache.java
new file mode 100644
index 0000000..0ddca4e
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalCredentialsCache.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.app.external;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.asterix.common.external.IExternalCredentialsCache;
+import org.apache.asterix.common.metadata.MetadataConstants;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.aws.s3.S3Constants;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hyracks.util.Span;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
+
+public class ExternalCredentialsCache implements IExternalCredentialsCache {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+    private final ConcurrentMap<String, Pair<Span, Object>> cache = new ConcurrentHashMap<>();
+
+    public ExternalCredentialsCache() {
+    }
+
+    @Override
+    public synchronized Object getCredentials(Map<String, String> configuration) {
+        String name = getName(configuration);
+        if (cache.containsKey(name) && !needsRefresh(cache.get(name).getLeft())) {
+            return cache.get(name).getRight();
+        }
+        return null;
+    }
+
+    @Override
+    public synchronized void updateCache(Map<String, String> configuration, Map<String, String> credentials) {
+        String type = configuration.get(ExternalDataConstants.KEY_READER);
+        if (ExternalDataConstants.KEY_ADAPTER_NAME_AWS_S3.equalsIgnoreCase(type)) {
+            updateAwsCache(configuration, credentials);
+        }
+    }
+
+    @Override
+    public String getName(Map<String, String> configuration) {
+        String database = configuration.get(ExternalDataConstants.KEY_DATASET_DATABASE);
+        if (database == null) {
+            database = MetadataConstants.DEFAULT_DATABASE;
+        }
+        String dataverse = configuration.get(ExternalDataConstants.KEY_DATASET_DATAVERSE);
+        String dataset = configuration.get(ExternalDataConstants.KEY_DATASET);
+        return String.join(".", database, dataverse, dataset);
+    }
+
+    private void updateAwsCache(Map<String, String> configuration, Map<String, String> credentials) {
+        String accessKeyId = credentials.get(S3Constants.ACCESS_KEY_ID_FIELD_NAME);
+        String secretAccessKey = credentials.get(S3Constants.SECRET_ACCESS_KEY_FIELD_NAME);
+        String sessionToken = credentials.get(S3Constants.SESSION_TOKEN_FIELD_NAME);
+        doUpdateAwsCache(configuration, AwsSessionCredentials.create(accessKeyId, secretAccessKey, sessionToken));
+    }
+
+    private void doUpdateAwsCache(Map<String, String> configuration, AwsSessionCredentials credentials) {
+        // TODO(htowaileb): Set default expiration value
+        String name = getName(configuration);
+        cache.put(name, Pair.of(Span.start(15, TimeUnit.MINUTES), credentials));
+        LOGGER.info("Received and cached new credentials for {}", name);
+    }
+
+    /**
+     * Refresh if the remaining time is half or less than the total expiration time
+     *
+     * @param span expiration span
+     * @return true if the remaining time is half or less than the total expiration time, false otherwise
+     */
+    private boolean needsRefresh(Span span) {
+        // TODO(htowaileb): At what % (and should be configurable?) do we decide it's better to refresh credentials
+        return (double) span.remaining(TimeUnit.MINUTES) / span.getSpan(TimeUnit.MINUTES) < 0.5;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalCredentialsCacheUpdater.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalCredentialsCacheUpdater.java
new file mode 100644
index 0000000..f07caaa
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalCredentialsCacheUpdater.java
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.app.external;
+
+import static org.apache.asterix.app.message.ExecuteStatementRequestMessage.DEFAULT_NC_TIMEOUT_MILLIS;
+import static org.apache.asterix.common.api.IClusterManagementWork.ClusterState.ACTIVE;
+import static org.apache.asterix.common.api.IClusterManagementWork.ClusterState.REBALANCE_REQUIRED;
+import static org.apache.asterix.common.exceptions.ErrorCode.REJECT_BAD_CLUSTER_STATE;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.asterix.app.message.RefreshAwsCredentialsRequest;
+import org.apache.asterix.app.message.RefreshAwsCredentialsResponse;
+import org.apache.asterix.app.message.UpdateAwsCredentialsCacheRequest;
+import org.apache.asterix.common.api.IApplicationContext;
+import org.apache.asterix.common.api.IClusterManagementWork;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.common.external.IExternalCredentialsCache;
+import org.apache.asterix.common.external.IExternalCredentialsCacheUpdater;
+import org.apache.asterix.common.messaging.api.MessageFuture;
+import org.apache.asterix.external.util.aws.s3.S3AuthUtils;
+import org.apache.asterix.external.util.aws.s3.S3Constants;
+import org.apache.asterix.messaging.CCMessageBroker;
+import org.apache.asterix.messaging.NCMessageBroker;
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
+
+public class ExternalCredentialsCacheUpdater implements IExternalCredentialsCacheUpdater {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+    private final IApplicationContext appCtx;
+
+    public ExternalCredentialsCacheUpdater(IApplicationContext appCtx) {
+        this.appCtx = appCtx;
+    }
+
+    @Override
+    public synchronized Object generateAndCacheCredentials(Map<String, String> configuration)
+            throws HyracksDataException, CompilationException {
+        IExternalCredentialsCache cache = appCtx.getExternalCredentialsCache();
+        Object credentials = cache.getCredentials(configuration);
+        if (credentials != null) {
+            return credentials;
+        }
+
+        /*
+         * if we are the CC, generate new creds and ask all NCs to update their cache
+         * if we are the NC, send a message to the CC to generate new creds and ask all NCs to update their cache
+         */
+        String name = cache.getName(configuration);
+        if (appCtx instanceof ICcApplicationContext) {
+            ICcApplicationContext ccAppCtx = (ICcApplicationContext) appCtx;
+            IClusterManagementWork.ClusterState state = ccAppCtx.getClusterStateManager().getState();
+            if (!(state == ACTIVE || state == REBALANCE_REQUIRED)) {
+                throw new RuntimeDataException(REJECT_BAD_CLUSTER_STATE, state);
+            }
+
+            String accessKeyId;
+            String secretAccessKey;
+            String sessionToken;
+            Map<String, String> credentialsMap = new HashMap<>();
+            try {
+                LOGGER.info("attempting to update credentials for {}", name);
+                AwsCredentialsProvider newCredentials = S3AuthUtils.assumeRoleAndGetCredentials(configuration);
+                LOGGER.info("updated credentials successfully for {}", name);
+                AwsSessionCredentials sessionCredentials = (AwsSessionCredentials) newCredentials.resolveCredentials();
+                accessKeyId = sessionCredentials.accessKeyId();
+                secretAccessKey = sessionCredentials.secretAccessKey();
+                sessionToken = sessionCredentials.sessionToken();
+            } catch (CompilationException ex) {
+                LOGGER.info("failed to refresh credentials for {}", name, ex);
+                throw ex;
+            }
+
+            // credentials need refreshing
+            credentialsMap.put(S3Constants.ACCESS_KEY_ID_FIELD_NAME, accessKeyId);
+            credentialsMap.put(S3Constants.SECRET_ACCESS_KEY_FIELD_NAME, secretAccessKey);
+            credentialsMap.put(S3Constants.SESSION_TOKEN_FIELD_NAME, sessionToken);
+
+            // request all NCs to update their credentials cache with the latest creds
+            updateNcsCredentialsCache(ccAppCtx, name, credentialsMap, configuration);
+            cache.updateCache(configuration, credentialsMap);
+            credentials = AwsSessionCredentials.create(accessKeyId, secretAccessKey, sessionToken);
+        } else {
+            NCMessageBroker broker = (NCMessageBroker) appCtx.getServiceContext().getMessageBroker();
+            MessageFuture messageFuture = broker.registerMessageFuture();
+            String nodeId = ((INCServiceContext) appCtx.getServiceContext()).getNodeId();
+            long futureId = messageFuture.getFutureId();
+            RefreshAwsCredentialsRequest request = new RefreshAwsCredentialsRequest(nodeId, futureId, configuration);
+            try {
+                LOGGER.info("no valid credentials found for {}, requesting credentials from CC", name);
+                broker.sendMessageToPrimaryCC(request);
+                RefreshAwsCredentialsResponse response = (RefreshAwsCredentialsResponse) messageFuture
+                        .get(DEFAULT_NC_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS);
+                if (response.getFailure() != null) {
+                    throw HyracksDataException.create(response.getFailure());
+                }
+                credentials = AwsSessionCredentials.create(response.getAccessKeyId(), response.getSecretAccessKey(),
+                        response.getSessionToken());
+            } catch (Exception ex) {
+                LOGGER.info("failed to refresh credentials for {}", name, ex);
+                throw HyracksDataException.create(ex);
+            } finally {
+                broker.deregisterMessageFuture(futureId);
+            }
+        }
+        return credentials;
+    }
+
+    private void updateNcsCredentialsCache(ICcApplicationContext appCtx, String name, Map<String, String> credentials,
+            Map<String, String> configuration) throws HyracksDataException {
+        final List<String> ncs = new ArrayList<>(appCtx.getClusterStateManager().getParticipantNodes());
+        CCMessageBroker broker = (CCMessageBroker) appCtx.getServiceContext().getMessageBroker();
+        UpdateAwsCredentialsCacheRequest request = new UpdateAwsCredentialsCacheRequest(configuration, credentials);
+
+        try {
+            LOGGER.info("requesting all NCs to update their credentials for {}", name);
+            for (String nc : ncs) {
+                broker.sendApplicationMessageToNC(request, nc);
+            }
+        } catch (Exception e) {
+            LOGGER.info("failed to send message to nc", e);
+            throw HyracksDataException.create(e);
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java
index 45776a4..470ec34 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java
@@ -85,6 +85,9 @@
     }
 
     protected boolean invalidArgs(List<Mutable<ILogicalExpression>> args) {
+        if (functionId.getArity() == FunctionIdentifier.VARARGS) {
+            return false;
+        }
         return args.size() != functionId.getArity();
     }
 
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/collectionsize/StorageSizeDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/collectionsize/StorageSizeDatasource.java
new file mode 100644
index 0000000..03d7bed
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/collectionsize/StorageSizeDatasource.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.app.function.collectionsize;
+
+import java.util.Objects;
+
+import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.metadata.api.IDatasourceFunction;
+import org.apache.asterix.metadata.declared.DataSourceId;
+import org.apache.asterix.metadata.declared.FunctionDataSource;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+
+public class StorageSizeDatasource extends FunctionDataSource {
+
+    private static final DataSourceId STORAGE_SIZE_DATASOURCE_ID =
+            new DataSourceId(StorageSizeRewriter.STORAGE_SIZE.getDatabase(),
+                    FunctionSignature.getDataverseName(StorageSizeRewriter.STORAGE_SIZE),
+                    StorageSizeRewriter.STORAGE_SIZE.getName());
+    private final String database;
+    private final DataverseName dataverse;
+    private final String collection;
+    private final String index;
+
+    StorageSizeDatasource(INodeDomain domain, String database, DataverseName dataverse, String collection, String index)
+            throws AlgebricksException {
+        super(STORAGE_SIZE_DATASOURCE_ID, StorageSizeRewriter.STORAGE_SIZE, domain);
+        this.database = database;
+        this.dataverse = dataverse;
+        this.collection = collection;
+        this.index = index;
+    }
+
+    public String getDatabase() {
+        return database;
+    }
+
+    public DataverseName getDataverse() {
+        return dataverse;
+    }
+
+    public String getCollection() {
+        return collection;
+    }
+
+    public String getIndex() {
+        return index;
+    }
+
+    @Override
+    protected IDatasourceFunction createFunction(MetadataProvider metadataProvider,
+            AlgebricksAbsolutePartitionConstraint locations) {
+        return new StorageSizeFunction(AlgebricksAbsolutePartitionConstraint.randomLocation(locations.getLocations()),
+                database, dataverse, collection, index);
+    }
+
+    @Override
+    protected boolean sameFunctionDatasource(FunctionDataSource other) {
+        if (!Objects.equals(this.functionId, other.getFunctionId())) {
+            return false;
+        }
+        StorageSizeDatasource that = (StorageSizeDatasource) other;
+        return Objects.equals(this.database, that.getDatabase()) && Objects.equals(this.dataverse, that.getDataverse())
+                && Objects.equals(this.collection, that.getCollection()) && Objects.equals(this.index, that.getIndex());
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/collectionsize/StorageSizeFunction.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/collectionsize/StorageSizeFunction.java
new file mode 100644
index 0000000..9f165bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/collectionsize/StorageSizeFunction.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.app.function.collectionsize;
+
+import static org.apache.asterix.app.message.ExecuteStatementRequestMessage.DEFAULT_NC_TIMEOUT_MILLIS;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.asterix.app.message.CalculateStorageSizeRequestMessage;
+import org.apache.asterix.app.message.CalculateStorageSizeResponseMessage;
+import org.apache.asterix.common.messaging.api.INCMessageBroker;
+import org.apache.asterix.common.messaging.api.MessageFuture;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.metadata.declared.AbstractDatasourceFunction;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class StorageSizeFunction extends AbstractDatasourceFunction {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+    private static final long serialVersionUID = 1L;
+
+    private final String database;
+    private final DataverseName dataverse;
+    private final String collection;
+    private final String index;
+
+    StorageSizeFunction(AlgebricksAbsolutePartitionConstraint locations, String database, DataverseName dataverse,
+            String collection, String index) {
+        super(locations);
+        this.database = database;
+        this.dataverse = dataverse;
+        this.collection = collection;
+        this.index = index;
+    }
+
+    @Override
+    public IRecordReader<char[]> createRecordReader(IHyracksTaskContext ctx, int partition)
+            throws HyracksDataException {
+        INCServiceContext serviceCtx = ctx.getJobletContext().getServiceContext();
+        INCMessageBroker messageBroker = (INCMessageBroker) serviceCtx.getMessageBroker();
+        MessageFuture messageFuture = messageBroker.registerMessageFuture();
+        long futureId = messageFuture.getFutureId();
+        CalculateStorageSizeRequestMessage request = new CalculateStorageSizeRequestMessage(serviceCtx.getNodeId(),
+                futureId, database, dataverse, collection, index);
+        try {
+            messageBroker.sendMessageToPrimaryCC(request);
+            CalculateStorageSizeResponseMessage response = (CalculateStorageSizeResponseMessage) messageFuture
+                    .get(DEFAULT_NC_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS);
+            if (response.getFailure() != null) {
+                throw HyracksDataException.create(response.getFailure());
+            }
+            return new StorageSizeReader(response.getSize());
+        } catch (Exception e) {
+            LOGGER.info("Could not calculate collection size", e);
+            throw HyracksDataException.create(e);
+        } finally {
+            messageBroker.deregisterMessageFuture(futureId);
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/collectionsize/StorageSizeReader.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/collectionsize/StorageSizeReader.java
new file mode 100644
index 0000000..733d39c
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/collectionsize/StorageSizeReader.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.app.function.collectionsize;
+
+import java.io.IOException;
+
+import org.apache.asterix.app.function.FunctionReader;
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.input.record.CharArrayRecord;
+
+public class StorageSizeReader extends FunctionReader {
+
+    private final long size;
+    private final CharArrayRecord record;
+    private boolean hasNext = true;
+
+    StorageSizeReader(long size) {
+        this.size = size;
+        record = new CharArrayRecord();
+    }
+
+    @Override
+    public boolean hasNext() throws IOException {
+        return hasNext;
+    }
+
+    @Override
+    public IRawRecord<char[]> next() throws IOException {
+        hasNext = false;
+        record.reset();
+        String result = "{\"size\":" + size + "}";
+        record.append(result.toCharArray());
+        record.endRecord();
+        return record;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/collectionsize/StorageSizeRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/collectionsize/StorageSizeRewriter.java
new file mode 100644
index 0000000..d551b7d
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/collectionsize/StorageSizeRewriter.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.app.function.collectionsize;
+
+import static org.apache.asterix.common.exceptions.ErrorCode.TYPE_MISMATCH_FUNCTION;
+
+import java.util.List;
+
+import org.apache.asterix.app.function.FunctionRewriter;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.functions.FunctionConstants;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.metadata.declared.FunctionDataSource;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.exceptions.ExceptionUtil;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+/**
+ * This function takes a collection's fully qualified name (database.scope.collection) and returns the collection's size
+ */
+
+public class StorageSizeRewriter extends FunctionRewriter {
+
+    public static final FunctionIdentifier STORAGE_SIZE =
+            FunctionConstants.newAsterix("storage-size", FunctionIdentifier.VARARGS);
+    public static final StorageSizeRewriter INSTANCE = new StorageSizeRewriter(STORAGE_SIZE);
+
+    private StorageSizeRewriter(FunctionIdentifier functionId) {
+        super(functionId);
+    }
+
+    @Override
+    protected FunctionDataSource toDatasource(IOptimizationContext context, AbstractFunctionCallExpression function)
+            throws AlgebricksException {
+
+        if (function.getArguments().size() < 3 || function.getArguments().size() > 4) {
+            throw new CompilationException(ErrorCode.COMPILATION_INVALID_NUM_OF_ARGS, STORAGE_SIZE.getName());
+        }
+
+        verifyArgs(function.getArguments());
+        ILogicalExpression databaseExpr = function.getArguments().get(0).getValue();
+        ILogicalExpression scopeExpr = function.getArguments().get(1).getValue();
+        ILogicalExpression collectionExpr = function.getArguments().get(2).getValue();
+        ILogicalExpression indexExpr = null;
+        if (function.getArguments().size() == 4) {
+            indexExpr = function.getArguments().get(3).getValue();
+        }
+
+        String database = ConstantExpressionUtil.getStringConstant(databaseExpr);
+        DataverseName dataverse =
+                DataverseName.createSinglePartName(ConstantExpressionUtil.getStringConstant(scopeExpr));
+        String collection = ConstantExpressionUtil.getStringConstant(collectionExpr);
+        String index = indexExpr != null ? ConstantExpressionUtil.getStringConstant(indexExpr) : null;
+
+        return new StorageSizeDatasource(context.getComputationNodeDomain(), database, dataverse, collection, index);
+    }
+
+    private void verifyArgs(List<Mutable<ILogicalExpression>> args) throws CompilationException {
+        for (int i = 0; i < args.size(); i++) {
+            ConstantExpression expr = (ConstantExpression) args.get(i).getValue();
+            AsterixConstantValue value = (AsterixConstantValue) expr.getValue();
+            ATypeTag type = value.getObject().getType().getTypeTag();
+            if (type != ATypeTag.STRING) {
+                throw new CompilationException(TYPE_MISMATCH_FUNCTION, STORAGE_SIZE.getName(),
+                        ExceptionUtil.indexToPosition(i), ATypeTag.STRING, type);
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CalculateStorageSizeRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CalculateStorageSizeRequestMessage.java
new file mode 100644
index 0000000..1163e5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CalculateStorageSizeRequestMessage.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.app.message;
+
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.messaging.api.ICcAddressedMessage;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.messaging.CCMessageBroker;
+import org.apache.asterix.utils.StorageUtil;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class CalculateStorageSizeRequestMessage implements ICcAddressedMessage {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+    private static final long serialVersionUID = 1L;
+    private static final int FAILED_CALCULATED_SIZE = -1;
+    private final String nodeId;
+    private final long reqId;
+    private final String database;
+    private final DataverseName dataverse;
+    private final String collection;
+    private final String index;
+
+    public CalculateStorageSizeRequestMessage(String nodeId, long reqId, String database, DataverseName dataverse,
+            String collection, String index) {
+        this.nodeId = nodeId;
+        this.reqId = reqId;
+        this.database = database;
+        this.dataverse = dataverse;
+        this.collection = collection;
+        this.index = index;
+    }
+
+    @Override
+    public void handle(ICcApplicationContext appCtx) throws HyracksDataException {
+        CCMessageBroker messageBroker = (CCMessageBroker) appCtx.getServiceContext().getMessageBroker();
+
+        try {
+            long size = StorageUtil.getCollectionSize(appCtx, database, dataverse, collection, index);
+            CalculateStorageSizeResponseMessage response =
+                    new CalculateStorageSizeResponseMessage(this.reqId, size, null);
+            messageBroker.sendApplicationMessageToNC(response, nodeId);
+        } catch (Exception ex) {
+            LOGGER.info("Failed to process request", ex);
+            try {
+                CalculateStorageSizeResponseMessage response =
+                        new CalculateStorageSizeResponseMessage(this.reqId, FAILED_CALCULATED_SIZE, ex);
+                messageBroker.sendApplicationMessageToNC(response, nodeId);
+            } catch (Exception ex2) {
+                LOGGER.info("Failed to process request", ex2);
+                throw HyracksDataException.create(ex2);
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CalculateStorageSizeResponseMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CalculateStorageSizeResponseMessage.java
new file mode 100644
index 0000000..aa7989e
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CalculateStorageSizeResponseMessage.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.app.message;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.messaging.api.INcAddressedMessage;
+import org.apache.asterix.common.messaging.api.MessageFuture;
+import org.apache.asterix.messaging.NCMessageBroker;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class CalculateStorageSizeResponseMessage implements INcAddressedMessage {
+
+    private static final long serialVersionUID = 1L;
+    private final long reqId;
+    private final long size;
+    private final Throwable failure;
+
+    public CalculateStorageSizeResponseMessage(long reqId, long size, Throwable failure) {
+        this.reqId = reqId;
+        this.size = size;
+        this.failure = failure;
+    }
+
+    @Override
+    public void handle(INcApplicationContext appCtx) throws HyracksDataException, InterruptedException {
+        NCMessageBroker mb = (NCMessageBroker) appCtx.getServiceContext().getMessageBroker();
+        MessageFuture future = mb.deregisterMessageFuture(reqId);
+        if (future != null) {
+            future.complete(this);
+        }
+    }
+
+    public long getSize() {
+        return size;
+    }
+
+    public Throwable getFailure() {
+        return failure;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/RefreshAwsCredentialsRequest.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/RefreshAwsCredentialsRequest.java
new file mode 100644
index 0000000..32de92d
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/RefreshAwsCredentialsRequest.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.app.message;
+
+import java.util.Map;
+
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.external.IExternalCredentialsCacheUpdater;
+import org.apache.asterix.common.messaging.api.ICcAddressedMessage;
+import org.apache.asterix.messaging.CCMessageBroker;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
+
+public class RefreshAwsCredentialsRequest implements ICcAddressedMessage {
+    private static final long serialVersionUID = 1L;
+    private static final Logger LOGGER = LogManager.getLogger();
+
+    private final String nodeId;
+    private final long reqId;
+    private final Map<String, String> configuration;
+
+    public RefreshAwsCredentialsRequest(String nodeId, long reqId, Map<String, String> configuration) {
+        this.nodeId = nodeId;
+        this.reqId = reqId;
+        this.configuration = configuration;
+    }
+
+    @Override
+    public final void handle(ICcApplicationContext appCtx) throws HyracksDataException {
+        try {
+            IExternalCredentialsCacheUpdater cacheUpdater = appCtx.getExternalCredentialsCacheUpdater();
+            Object credentials = cacheUpdater.generateAndCacheCredentials(configuration);
+            AwsSessionCredentials sessionCredentials = (AwsSessionCredentials) credentials;
+
+            // respond with the credentials
+            RefreshAwsCredentialsResponse response =
+                    new RefreshAwsCredentialsResponse(reqId, sessionCredentials.accessKeyId(),
+                            sessionCredentials.secretAccessKey(), sessionCredentials.sessionToken(), null);
+            respond(appCtx, response);
+        } catch (Exception e) {
+            LOGGER.info("failed to refresh credentials", e);
+            RefreshAwsCredentialsResponse response = new RefreshAwsCredentialsResponse(reqId, null, null, null, e);
+            respond(appCtx, response);
+        }
+    }
+
+    private void respond(ICcApplicationContext appCtx, RefreshAwsCredentialsResponse response)
+            throws HyracksDataException {
+        CCMessageBroker broker = (CCMessageBroker) appCtx.getServiceContext().getMessageBroker();
+        try {
+            broker.sendApplicationMessageToNC(response, nodeId);
+        } catch (Exception e) {
+            LOGGER.info("failed to send reply to nc", e);
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    @Override
+    public boolean isWhispered() {
+        return true;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/RefreshAwsCredentialsResponse.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/RefreshAwsCredentialsResponse.java
new file mode 100644
index 0000000..9ea0e11
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/RefreshAwsCredentialsResponse.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.app.message;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.messaging.api.INcAddressedMessage;
+import org.apache.asterix.common.messaging.api.MessageFuture;
+import org.apache.asterix.messaging.NCMessageBroker;
+
+public class RefreshAwsCredentialsResponse implements INcAddressedMessage {
+
+    private static final long serialVersionUID = 1L;
+    private final long reqId;
+    private final String accessKeyId;
+    private final String secretAccessKey;
+    private final String sessionToken;
+    private final Throwable failure;
+
+    public RefreshAwsCredentialsResponse(long reqId, String accessKeyId, String secretAccessKey, String sessionToken,
+            Throwable failure) {
+        this.reqId = reqId;
+        this.accessKeyId = accessKeyId;
+        this.secretAccessKey = secretAccessKey;
+        this.sessionToken = sessionToken;
+        this.failure = failure;
+    }
+
+    @Override
+    public void handle(INcApplicationContext appCtx) {
+        NCMessageBroker mb = (NCMessageBroker) appCtx.getServiceContext().getMessageBroker();
+        MessageFuture future = mb.deregisterMessageFuture(reqId);
+        if (future != null) {
+            future.complete(this);
+        }
+    }
+
+    public String getAccessKeyId() {
+        return accessKeyId;
+    }
+
+    public String getSecretAccessKey() {
+        return secretAccessKey;
+    }
+
+    public String getSessionToken() {
+        return sessionToken;
+    }
+
+    public Throwable getFailure() {
+        return failure;
+    }
+
+    @Override
+    public boolean isWhispered() {
+        return true;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/StorageSizeRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/StorageSizeRequestMessage.java
new file mode 100644
index 0000000..82b425b
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/StorageSizeRequestMessage.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.app.message;
+
+import java.util.function.Predicate;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.messaging.CcIdentifiedMessage;
+import org.apache.asterix.common.messaging.api.INcAddressedMessage;
+import org.apache.asterix.common.storage.ResourceReference;
+import org.apache.asterix.messaging.NCMessageBroker;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IIOManager;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class StorageSizeRequestMessage extends CcIdentifiedMessage implements INcAddressedMessage {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+    private static final long serialVersionUID = 1L;
+    private final long reqId;
+    private final String database;
+    private final String dataverse;
+    private final String collection;
+    private final String index;
+
+    public StorageSizeRequestMessage(long reqId, String database, String dataverse, String collection, String index) {
+        this.reqId = reqId;
+        this.database = database;
+        this.dataverse = dataverse;
+        this.collection = collection;
+        this.index = index;
+    }
+
+    @Override
+    public void handle(INcApplicationContext appCtx) throws HyracksDataException {
+        try {
+            Predicate<String> predicate = getPredicate();
+            IIOManager ioManager = appCtx.getPersistenceIoManager();
+            StorageSizeResponseMessage response =
+                    new StorageSizeResponseMessage(reqId, ioManager.getSize(predicate), null);
+            respond(appCtx, response);
+        } catch (Exception e) {
+            LOGGER.info("failed to get collection size", e);
+            StorageSizeResponseMessage response = new StorageSizeResponseMessage(reqId, 0, e);
+            respond(appCtx, response);
+        }
+    }
+
+    private Predicate<String> getPredicate() {
+        return path -> {
+            ResourceReference resourceReference = ResourceReference.of(path);
+            if (resourceReference.getDatabase().equals(database)
+                    && resourceReference.getDataverse().getCanonicalForm().equals(dataverse)
+                    && resourceReference.getDataset().equals(collection)) {
+                if (index != null) {
+                    return resourceReference.getIndex().equals(index);
+                }
+                return true;
+            }
+            return false;
+        };
+    }
+
+    private void respond(INcApplicationContext appCtx, StorageSizeResponseMessage response)
+            throws HyracksDataException {
+        NCMessageBroker messageBroker = (NCMessageBroker) appCtx.getServiceContext().getMessageBroker();
+        try {
+            messageBroker.sendMessageToPrimaryCC(response);
+        } catch (Exception e) {
+            LOGGER.info("failed to send collection size to cc", e);
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    @Override
+    public boolean isWhispered() {
+        return true;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/StorageSizeResponseMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/StorageSizeResponseMessage.java
new file mode 100644
index 0000000..09850f1
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/StorageSizeResponseMessage.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.app.message;
+
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.messaging.api.ICCMessageBroker;
+import org.apache.asterix.common.messaging.api.ICCMessageBroker.ResponseState;
+import org.apache.asterix.common.messaging.api.ICcAddressedMessage;
+import org.apache.asterix.common.messaging.api.INcResponse;
+import org.apache.commons.lang3.tuple.MutablePair;
+
+public class StorageSizeResponseMessage implements ICcAddressedMessage, INcResponse {
+
+    private static final long serialVersionUID = 1L;
+    private final long reqId;
+    private final long size;
+    private final Throwable failure;
+
+    public StorageSizeResponseMessage(long reqId, long size, Throwable failure) {
+        this.reqId = reqId;
+        this.size = size;
+        this.failure = failure;
+    }
+
+    @Override
+    public void handle(ICcApplicationContext appCtx) {
+        ICCMessageBroker broker = (ICCMessageBroker) appCtx.getServiceContext().getMessageBroker();
+        broker.respond(reqId, this);
+    }
+
+    @Override
+    public void setResult(MutablePair<ResponseState, Object> result) {
+        if (failure != null) {
+            result.setLeft(ResponseState.FAILURE);
+            result.setRight(failure);
+            return;
+        }
+        setResponse(result);
+    }
+
+    private void setResponse(MutablePair<ResponseState, Object> result) {
+        switch (result.getKey()) {
+            case SUCCESS:
+                long currentSize = (long) result.getValue();
+                result.setValue(currentSize + size);
+                break;
+            case UNINITIALIZED:
+                result.setLeft(ResponseState.SUCCESS);
+                result.setValue(size);
+                break;
+            default:
+                break;
+        }
+    }
+
+    @Override
+    public boolean isWhispered() {
+        return true;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/UpdateAwsCredentialsCacheRequest.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/UpdateAwsCredentialsCacheRequest.java
new file mode 100644
index 0000000..44d4c21
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/UpdateAwsCredentialsCacheRequest.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.app.message;
+
+import java.util.Map;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.messaging.api.INcAddressedMessage;
+
+public class UpdateAwsCredentialsCacheRequest implements INcAddressedMessage {
+
+    private static final long serialVersionUID = 1L;
+    private final Map<String, String> configuration;
+    private final Map<String, String> credentials;
+
+    public UpdateAwsCredentialsCacheRequest(Map<String, String> configuration, Map<String, String> credentials) {
+        this.configuration = configuration;
+        this.credentials = credentials;
+    }
+
+    @Override
+    public void handle(INcApplicationContext appCtx) {
+        appCtx.getExternalCredentialsCache().updateCache(configuration, credentials);
+    }
+
+    @Override
+    public boolean isWhispered() {
+        return true;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
index 7da3838..8c2a0ab 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
@@ -32,6 +32,8 @@
 import java.util.concurrent.ExecutorService;
 
 import org.apache.asterix.active.ActiveManager;
+import org.apache.asterix.app.external.ExternalCredentialsCache;
+import org.apache.asterix.app.external.ExternalCredentialsCacheUpdater;
 import org.apache.asterix.app.result.ResultReader;
 import org.apache.asterix.cloud.CloudConfigurator;
 import org.apache.asterix.cloud.LocalPartitionBootstrapper;
@@ -63,6 +65,8 @@
 import org.apache.asterix.common.context.DiskWriteRateLimiterProvider;
 import org.apache.asterix.common.context.GlobalVirtualBufferCache;
 import org.apache.asterix.common.context.IStorageComponentProvider;
+import org.apache.asterix.common.external.IExternalCredentialsCache;
+import org.apache.asterix.common.external.IExternalCredentialsCacheUpdater;
 import org.apache.asterix.common.library.ILibraryManager;
 import org.apache.asterix.common.replication.IReplicationChannel;
 import org.apache.asterix.common.replication.IReplicationManager;
@@ -186,6 +190,8 @@
     private final INamespacePathResolver namespacePathResolver;
     private final INamespaceResolver namespaceResolver;
     private IDiskCacheMonitoringService diskCacheService;
+    protected IExternalCredentialsCache externalCredentialsCache;
+    protected IExternalCredentialsCacheUpdater externalCredentialsCacheUpdater;
 
     public NCAppRuntimeContext(INCServiceContext ncServiceContext, NCExtensionManager extensionManager,
             IPropertiesFactory propertiesFactory, INamespaceResolver namespaceResolver,
@@ -210,6 +216,8 @@
         cacheManager = new CacheManager();
         this.namespacePathResolver = namespacePathResolver;
         this.namespaceResolver = namespaceResolver;
+        this.externalCredentialsCache = new ExternalCredentialsCache();
+        this.externalCredentialsCacheUpdater = new ExternalCredentialsCacheUpdater(this);
     }
 
     @Override
@@ -748,4 +756,14 @@
         return isCloudDeployment() ? storageProperties.getStoragePartitionsCount()
                 : ncServiceContext.getIoManager().getIODevices().size();
     }
+
+    @Override
+    public IExternalCredentialsCache getExternalCredentialsCache() {
+        return externalCredentialsCache;
+    }
+
+    @Override
+    public IExternalCredentialsCacheUpdater getExternalCredentialsCacheUpdater() {
+        return externalCredentialsCacheUpdater;
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index 0c736a2..ed93838 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -27,6 +27,7 @@
 import java.io.InputStream;
 import java.rmi.RemoteException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Date;
@@ -109,6 +110,7 @@
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.ExternalDataUtils;
 import org.apache.asterix.external.util.WriterValidationUtil;
+import org.apache.asterix.external.writer.printer.parquet.SchemaConverterVisitor;
 import org.apache.asterix.lang.common.base.IQueryRewriter;
 import org.apache.asterix.lang.common.base.IReturningStatement;
 import org.apache.asterix.lang.common.base.IRewriterFactory;
@@ -189,6 +191,7 @@
 import org.apache.asterix.metadata.entities.DatasourceAdapter;
 import org.apache.asterix.metadata.entities.Datatype;
 import org.apache.asterix.metadata.entities.Dataverse;
+import org.apache.asterix.metadata.entities.EntityDetails;
 import org.apache.asterix.metadata.entities.ExternalDatasetDetails;
 import org.apache.asterix.metadata.entities.Feed;
 import org.apache.asterix.metadata.entities.FeedConnection;
@@ -204,6 +207,7 @@
 import org.apache.asterix.metadata.entities.ViewDetails;
 import org.apache.asterix.metadata.feeds.FeedMetadataUtil;
 import org.apache.asterix.metadata.functions.ExternalFunctionCompilerUtil;
+import org.apache.asterix.metadata.utils.Creator;
 import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.metadata.utils.IndexUtil;
 import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
@@ -213,6 +217,7 @@
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.BuiltinTypeMap;
 import org.apache.asterix.om.types.IAType;
@@ -383,16 +388,20 @@
                         activeNamespace = handleUseDataverseStatement(metadataProvider, stmt);
                         break;
                     case CREATE_DATABASE:
-                        handleCreateDatabaseStatement(metadataProvider, stmt, requestParameters);
+                        handleCreateDatabaseStatement(metadataProvider, stmt, requestParameters,
+                                Creator.DEFAULT_CREATOR);
                         break;
                     case CREATE_DATAVERSE:
-                        handleCreateDataverseStatement(metadataProvider, stmt, requestParameters);
+                        handleCreateDataverseStatement(metadataProvider, stmt, requestParameters,
+                                Creator.DEFAULT_CREATOR);
                         break;
                     case DATASET_DECL:
-                        handleCreateDatasetStatement(metadataProvider, stmt, hcc, requestParameters);
+                        handleCreateDatasetStatement(metadataProvider, stmt, hcc, requestParameters,
+                                Creator.DEFAULT_CREATOR);
                         break;
                     case CREATE_INDEX:
-                        handleCreateIndexStatement(metadataProvider, stmt, hcc, requestParameters);
+                        handleCreateIndexStatement(metadataProvider, stmt, hcc, requestParameters,
+                                Creator.DEFAULT_CREATOR);
                         break;
                     case CREATE_FULL_TEXT_FILTER:
                         handleCreateFullTextFilterStatement(metadataProvider, stmt);
@@ -437,7 +446,8 @@
                         handleAdapterDropStatement(metadataProvider, stmt);
                         break;
                     case CREATE_FUNCTION:
-                        handleCreateFunctionStatement(metadataProvider, stmt, stmtRewriter, requestParameters);
+                        handleCreateFunctionStatement(metadataProvider, stmt, stmtRewriter, requestParameters,
+                                Creator.DEFAULT_CREATOR);
                         break;
                     case FUNCTION_DROP:
                         handleFunctionDropStatement(metadataProvider, stmt, requestParameters);
@@ -449,13 +459,15 @@
                         handleLibraryDropStatement(metadataProvider, stmt, hcc, requestParameters);
                         break;
                     case CREATE_SYNONYM:
-                        handleCreateSynonymStatement(metadataProvider, stmt, requestParameters);
+                        handleCreateSynonymStatement(metadataProvider, stmt, requestParameters,
+                                Creator.DEFAULT_CREATOR);
                         break;
                     case SYNONYM_DROP:
                         handleDropSynonymStatement(metadataProvider, stmt, requestParameters);
                         break;
                     case CREATE_VIEW:
-                        handleCreateViewStatement(metadataProvider, stmt, stmtRewriter, requestParameters);
+                        handleCreateViewStatement(metadataProvider, stmt, stmtRewriter, requestParameters,
+                                Creator.DEFAULT_CREATOR);
                         break;
                     case VIEW_DROP:
                         handleViewDropStatement(metadataProvider, stmt, requestParameters);
@@ -656,7 +668,7 @@
     }
 
     protected void handleCreateDatabaseStatement(MetadataProvider metadataProvider, Statement stmt,
-            IRequestParameters requestParameters) throws Exception {
+            IRequestParameters requestParameters, Creator creator) throws Exception {
         CreateDatabaseStatement stmtCreateDatabase = (CreateDatabaseStatement) stmt;
         String database = stmtCreateDatabase.getDatabaseName().getValue();
         metadataProvider.validateDatabaseName(database, stmt.getSourceLocation());
@@ -665,14 +677,14 @@
         }
         lockUtil.createDatabaseBegin(lockManager, metadataProvider.getLocks(), database);
         try {
-            doCreateDatabaseStatement(metadataProvider, stmtCreateDatabase, requestParameters);
+            doCreateDatabaseStatement(metadataProvider, stmtCreateDatabase, requestParameters, creator);
         } finally {
             metadataProvider.getLocks().unlock();
         }
     }
 
     protected boolean doCreateDatabaseStatement(MetadataProvider mdProvider, CreateDatabaseStatement stmtCreateDatabase,
-            IRequestParameters requestParameters) throws Exception {
+            IRequestParameters requestParameters, Creator creator) throws Exception {
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         mdProvider.setMetadataTxnContext(mdTxnCtx);
         try {
@@ -687,8 +699,10 @@
                             databaseName);
                 }
             }
+
+            beforeTxnCommit(mdProvider, creator, EntityDetails.newDatabase(databaseName));
             MetadataManager.INSTANCE.addDatabase(mdTxnCtx,
-                    new Database(databaseName, false, MetadataUtil.PENDING_NO_OP));
+                    new Database(databaseName, false, MetadataUtil.PENDING_NO_OP, creator));
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             return true;
         } catch (Exception e) {
@@ -698,7 +712,7 @@
     }
 
     protected void handleCreateDataverseStatement(MetadataProvider metadataProvider, Statement stmt,
-            IRequestParameters requestParameters) throws Exception {
+            IRequestParameters requestParameters, Creator creator) throws Exception {
         CreateDataverseStatement stmtCreateDataverse = (CreateDataverseStatement) stmt;
         DataverseName dvName = stmtCreateDataverse.getDataverseName();
         String dbName = stmtCreateDataverse.getDatabaseName();
@@ -709,7 +723,7 @@
         }
         lockUtil.createDataverseBegin(lockManager, metadataProvider.getLocks(), dbName, dvName);
         try {
-            doCreateDataverseStatement(metadataProvider, stmtCreateDataverse, requestParameters);
+            doCreateDataverseStatement(metadataProvider, stmtCreateDataverse, requestParameters, creator);
         } finally {
             metadataProvider.getLocks().unlock();
         }
@@ -717,7 +731,8 @@
 
     @SuppressWarnings("squid:S00112")
     protected boolean doCreateDataverseStatement(MetadataProvider metadataProvider,
-            CreateDataverseStatement stmtCreateDataverse, IRequestParameters requestParameters) throws Exception {
+            CreateDataverseStatement stmtCreateDataverse, IRequestParameters requestParameters, Creator creator)
+            throws Exception {
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         try {
@@ -740,8 +755,9 @@
                             dvName);
                 }
             }
-            MetadataManager.INSTANCE.addDataverse(metadataProvider.getMetadataTxnContext(),
-                    new Dataverse(dbName, dvName, stmtCreateDataverse.getFormat(), MetadataUtil.PENDING_NO_OP));
+            beforeTxnCommit(metadataProvider, creator, EntityDetails.newDataverse(dbName, dvName));
+            MetadataManager.INSTANCE.addDataverse(metadataProvider.getMetadataTxnContext(), new Dataverse(dbName,
+                    dvName, stmtCreateDataverse.getFormat(), MetadataUtil.PENDING_NO_OP, creator));
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             return true;
         } catch (Exception e) {
@@ -788,7 +804,7 @@
     }
 
     public void handleCreateDatasetStatement(MetadataProvider metadataProvider, Statement stmt,
-            IHyracksClientConnection hcc, IRequestParameters requestParameters) throws Exception {
+            IHyracksClientConnection hcc, IRequestParameters requestParameters, Creator creator) throws Exception {
         DatasetDecl dd = (DatasetDecl) stmt;
         String datasetName = dd.getName().getValue();
         metadataProvider.validateDatabaseObjectName(dd.getNamespace(), datasetName, stmt.getSourceLocation());
@@ -838,7 +854,7 @@
         try {
             doCreateDatasetStatement(metadataProvider, dd, stmtActiveNamespace, datasetName, itemTypeNamespace,
                     itemTypeExpr, itemTypeName, metaItemTypeExpr, metaItemTypeNamespace, metaItemTypeName, hcc,
-                    requestParameters);
+                    requestParameters, creator);
             if (dd.getQuery() != null) {
                 final IResultSet resultSet = requestParameters.getResultSet();
                 final ResultDelivery resultDelivery = requestParameters.getResultProperties().getDelivery();
@@ -860,8 +876,8 @@
     protected Optional<? extends Dataset> doCreateDatasetStatement(MetadataProvider metadataProvider, DatasetDecl dd,
             Namespace namespace, String datasetName, Namespace itemTypeNamespace, TypeExpression itemTypeExpr,
             String itemTypeName, TypeExpression metaItemTypeExpr, Namespace metaItemTypeNamespace,
-            String metaItemTypeName, IHyracksClientConnection hcc, IRequestParameters requestParameters)
-            throws Exception {
+            String metaItemTypeName, IHyracksClientConnection hcc, IRequestParameters requestParameters,
+            Creator creator) throws Exception {
         DataverseName dataverseName = namespace.getDataverseName();
         String databaseName = namespace.getDatabaseName();
 
@@ -1034,7 +1050,7 @@
             dataset = (Dataset) createDataset(dd, databaseName, dataverseName, datasetName, itemTypeDatabaseName,
                     itemTypeDataverseName, itemTypeName, metaItemTypeDatabaseName, metaItemTypeDataverseName,
                     metaItemTypeName, dsType, compactionPolicy, compactionPolicyProperties, compressionScheme,
-                    datasetFormatInfo, datasetDetails, ngName);
+                    datasetFormatInfo, datasetDetails, ngName, creator);
             MetadataManager.INSTANCE.addDataset(metadataProvider.getMetadataTxnContext(), dataset);
 
             if (itemTypeIsInline) {
@@ -1068,6 +1084,8 @@
                     datasetName, requestParameters.isForceDropDataset());
             dataset.setPendingOp(MetadataUtil.PENDING_NO_OP);
             MetadataManager.INSTANCE.addDataset(metadataProvider.getMetadataTxnContext(), dataset);
+            beforeTxnCommit(metadataProvider, creator,
+                    EntityDetails.newDataset(databaseName, dataverseName, datasetName));
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
         } catch (Exception e) {
             if (bActiveTxn) {
@@ -1135,11 +1153,11 @@
             String metaItemTypeDatabase, DataverseName metaItemTypeDataverseName, String metaItemTypeName,
             DatasetType dsType, String compactionPolicy, Map<String, String> compactionPolicyProperties,
             String compressionScheme, DatasetFormatInfo datasetFormatInfo, IDatasetDetails datasetDetails,
-            String ngName) throws AlgebricksException {
+            String ngName, Creator creator) throws AlgebricksException {
         return new Dataset(database, dataverseName, datasetName, itemTypeDatabase, itemTypeDataverseName, itemTypeName,
                 metaItemTypeDatabase, metaItemTypeDataverseName, metaItemTypeName, ngName, compactionPolicy,
                 compactionPolicyProperties, datasetDetails, dd.getHints(), dsType, DatasetIdFactory.generateDatasetId(),
-                MetadataUtil.PENDING_ADD_OP, compressionScheme, datasetFormatInfo);
+                MetadataUtil.PENDING_ADD_OP, compressionScheme, datasetFormatInfo, creator);
     }
 
     protected Triple<Namespace, String, Boolean> extractDatasetItemTypeName(Namespace datasetNamespace,
@@ -1275,7 +1293,7 @@
     }
 
     public void handleCreateIndexStatement(MetadataProvider metadataProvider, Statement stmt,
-            IHyracksClientConnection hcc, IRequestParameters requestParameters) throws Exception {
+            IHyracksClientConnection hcc, IRequestParameters requestParameters, Creator creator) throws Exception {
         CreateIndexStatement stmtCreateIndex = (CreateIndexStatement) stmt;
         String datasetName = stmtCreateIndex.getDatasetName().getValue();
         String indexName = stmtCreateIndex.getIndexName().getValue();
@@ -1292,7 +1310,7 @@
                 fullTextConfigName);
         try {
             doCreateIndex(metadataProvider, stmtCreateIndex, databaseName, dataverseName, datasetName, hcc,
-                    requestParameters);
+                    requestParameters, creator);
         } finally {
             metadataProvider.getLocks().unlock();
         }
@@ -1300,7 +1318,7 @@
 
     protected void doCreateIndex(MetadataProvider metadataProvider, CreateIndexStatement stmtCreateIndex,
             String databaseName, DataverseName dataverseName, String datasetName, IHyracksClientConnection hcc,
-            IRequestParameters requestParameters) throws Exception {
+            IRequestParameters requestParameters, Creator creator) throws Exception {
         SourceLocation sourceLoc = stmtCreateIndex.getSourceLocation();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         boolean bActiveTxn = true;
@@ -1356,6 +1374,7 @@
             List<List<IAType>> indexFieldTypes = new ArrayList<>(indexedElementsCount);
             boolean hadUnnest = false;
             boolean overridesFieldTypes = false;
+            boolean isHeterogeneousIndex = false;
 
             // this set is used to detect duplicates in the specified keys in the create
             // index statement
@@ -1429,7 +1448,7 @@
                         projectTypeNullable = inputTypeNullable;
                         projectTypeMissable = inputTypeMissable;
                     } else if (inputTypePrime == null) {
-                        projectTypePrime = null; // ANY
+                        projectTypePrime = null;
                         projectTypeNullable = projectTypeMissable = true;
                     } else {
                         if (inputTypePrime.getTypeTag() != ATypeTag.OBJECT) {
@@ -1510,38 +1529,50 @@
                     }
 
                     if (fieldTypePrime == null) {
-                        if (projectPath != null) {
-                            String fieldName = LogRedactionUtil.userData(RecordUtil.toFullyQualifiedName(projectPath));
-                            throw new CompilationException(ErrorCode.COMPILATION_ERROR,
-                                    indexedElement.getSourceLocation(),
-                                    "cannot find type of field '" + fieldName + "'");
+                        if (indexType != IndexType.BTREE) {
+                            if (projectPath != null) {
+                                String fieldName =
+                                        LogRedactionUtil.userData(RecordUtil.toFullyQualifiedName(projectPath));
+                                throw new CompilationException(ErrorCode.COMPILATION_ERROR,
+                                        indexedElement.getSourceLocation(),
+                                        "cannot find type of field '" + fieldName + "'");
+                            }
+                            // projectPath == null should only be the case with array index having UNNESTs only
+                            if (indexedElement.hasUnnest()) {
+                                List<List<String>> unnestList = indexedElement.getUnnestList();
+                                List<String> arrayField = unnestList.get(unnestList.size() - 1);
+                                String fieldName =
+                                        LogRedactionUtil.userData(RecordUtil.toFullyQualifiedName(arrayField));
+                                throw new CompilationException(ErrorCode.COMPILATION_ERROR,
+                                        indexedElement.getSourceLocation(),
+                                        "cannot find type of elements of field '" + fieldName + "'");
+                            }
+                        } else {
+                            fieldTypePrime = BuiltinType.ANY;
+                            isHeterogeneousIndex = true;
+                            fieldTypeNullable = fieldTypeMissable = false;
                         }
-                        // projectPath == null should only be the case with array index having UNNESTs only
-                        if (indexedElement.hasUnnest()) {
-                            List<List<String>> unnestList = indexedElement.getUnnestList();
-                            List<String> arrayField = unnestList.get(unnestList.size() - 1);
-                            String fieldName = LogRedactionUtil.userData(RecordUtil.toFullyQualifiedName(arrayField));
-                            throw new CompilationException(ErrorCode.COMPILATION_ERROR,
-                                    indexedElement.getSourceLocation(),
-                                    "cannot find type of elements of field '" + fieldName + "'");
-                        }
-                        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
-                                indexedElement.getSourceLocation(), "cannot find type of field");
+                    }
+                    if (ATypeTag.ANY.equals(fieldTypePrime.getTypeTag()) && stmtCreateIndex.hasExcludeUnknownKey()) {
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, indexedElement.getSourceLocation(),
+                                "Cannot specify exclude/include unknown for untyped keys in the index definition.");
                     }
                     validateIndexFieldType(indexType, fieldTypePrime, projectPath, indexedElement.getSourceLocation());
 
                     IAType fieldType =
                             KeyFieldTypeUtil.makeUnknownableType(fieldTypePrime, fieldTypeNullable, fieldTypeMissable);
+                    if (isHeterogeneousIndex && !ATypeTag.ANY.equals(fieldType.getTypeTag())) {
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                                "Typed keys cannot be combined with untyped keys in the index definition.");
+                    }
                     fieldTypes.add(fieldType);
                 }
-
                 // Try to add the key & its source to the set of keys for duplicate detection.
                 if (!indexKeysSet.add(indexedElement.toIdentifier())) {
                     throw new AsterixException(ErrorCode.INDEX_ILLEGAL_REPETITIVE_FIELD,
                             indexedElement.getSourceLocation(),
                             LogRedactionUtil.userData(indexedElement.getProjectListDisplayForm()));
                 }
-
                 indexFieldTypes.add(fieldTypes);
             }
 
@@ -1625,10 +1656,11 @@
             }
 
             Index newIndex = new Index(databaseName, dataverseName, datasetName, indexName, indexType, indexDetails,
-                    stmtCreateIndex.isEnforced(), false, MetadataUtil.PENDING_ADD_OP);
+                    stmtCreateIndex.isEnforced(), false, MetadataUtil.PENDING_ADD_OP, creator);
 
             bActiveTxn = false; // doCreateIndexImpl() takes over the current transaction
-            doCreateIndexImpl(hcc, metadataProvider, ds, newIndex, jobFlags, sourceLoc);
+            EntityDetails entityDetails = EntityDetails.newIndex(databaseName, dataverseName, indexName);
+            doCreateIndexImpl(hcc, metadataProvider, ds, newIndex, jobFlags, sourceLoc, creator, entityDetails);
 
         } catch (Exception e) {
             if (bActiveTxn) {
@@ -1773,7 +1805,8 @@
     }
 
     private void doCreateIndexImpl(IHyracksClientConnection hcc, MetadataProvider metadataProvider, Dataset ds,
-            Index index, EnumSet<JobFlag> jobFlags, SourceLocation sourceLoc) throws Exception {
+            Index index, EnumSet<JobFlag> jobFlags, SourceLocation sourceLoc, Creator creator,
+            EntityDetails entityDetails) throws Exception {
         ProgressState progress = ProgressState.NO_PROGRESS;
         boolean bActiveTxn = true;
         MetadataTransactionContext mdTxnCtx = metadataProvider.getMetadataTxnContext();
@@ -1852,6 +1885,8 @@
                         "Failed to create job spec for creating index '" + ds.getDatasetName() + "."
                                 + index.getIndexName() + "'");
             }
+            beforeTxnCommit(metadataProvider, creator, entityDetails);
+
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
             progress = ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA;
@@ -2068,8 +2103,8 @@
             // second, inserting the database record with the PendingDropOp value into the 'Database' collection
             // Note: the delete operation fails if the database cannot be deleted due to metadata dependencies
             MetadataManager.INSTANCE.dropDatabase(mdTxnCtx, databaseName);
-            MetadataManager.INSTANCE.addDatabase(mdTxnCtx,
-                    new Database(databaseName, database.isSystemDatabase(), MetadataUtil.PENDING_DROP_OP));
+            MetadataManager.INSTANCE.addDatabase(mdTxnCtx, new Database(databaseName, database.isSystemDatabase(),
+                    MetadataUtil.PENDING_DROP_OP, database.getCreator()));
 
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
@@ -2254,8 +2289,8 @@
             // second, inserting the dataverse record with the PendingDropOp value into the DATAVERSE_DATASET
             // Note: the delete operation fails if the dataverse cannot be deleted due to metadata dependencies
             MetadataManager.INSTANCE.dropDataverse(mdTxnCtx, databaseName, dataverseName);
-            MetadataManager.INSTANCE.addDataverse(mdTxnCtx,
-                    new Dataverse(databaseName, dataverseName, dv.getDataFormat(), MetadataUtil.PENDING_DROP_OP));
+            MetadataManager.INSTANCE.addDataverse(mdTxnCtx, new Dataverse(databaseName, dataverseName,
+                    dv.getDataFormat(), MetadataUtil.PENDING_DROP_OP, dv.getCreator()));
 
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
@@ -2749,7 +2784,7 @@
     }
 
     public void handleCreateViewStatement(MetadataProvider metadataProvider, Statement stmt,
-            IStatementRewriter stmtRewriter, IRequestParameters requestParameters) throws Exception {
+            IStatementRewriter stmtRewriter, IRequestParameters requestParameters, Creator creator) throws Exception {
         CreateViewStatement cvs = (CreateViewStatement) stmt;
         String viewName = cvs.getViewName();
         metadataProvider.validateDatabaseObjectName(cvs.getNamespace(), viewName, stmt.getSourceLocation());
@@ -2783,7 +2818,7 @@
                 null, false, null, null, true, DatasetType.VIEW, null, metadataProvider);
         try {
             doCreateView(metadataProvider, cvs, databaseName, dataverseName, viewName, itemTypeDatabaseName,
-                    viewItemTypeDataverseName, viewItemTypeName, stmtRewriter, requestParameters);
+                    viewItemTypeDataverseName, viewItemTypeName, stmtRewriter, requestParameters, creator);
         } finally {
             metadataProvider.getLocks().unlock();
             metadataProvider.setDefaultNamespace(activeNamespace);
@@ -2793,7 +2828,7 @@
     protected CreateResult doCreateView(MetadataProvider metadataProvider, CreateViewStatement cvs, String databaseName,
             DataverseName dataverseName, String viewName, String itemTypeDatabaseName,
             DataverseName itemTypeDataverseName, String itemTypeName, IStatementRewriter stmtRewriter,
-            IRequestParameters requestParameters) throws Exception {
+            IRequestParameters requestParameters, Creator creator) throws Exception {
         SourceLocation sourceLoc = cvs.getSourceLocation();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
@@ -2959,10 +2994,10 @@
             ViewDetails viewDetails = new ViewDetails(cvs.getViewBody(), dependencies, cvs.getDefaultNull(),
                     primaryKeyFields, foreignKeys, datetimeFormat, dateFormat, timeFormat);
 
-            Dataset view =
-                    new Dataset(databaseName, dataverseName, viewName, itemTypeDatabaseName, itemTypeDataverseName,
-                            itemTypeName, MetadataConstants.METADATA_NODEGROUP_NAME, "", Collections.emptyMap(),
-                            viewDetails, Collections.emptyMap(), DatasetType.VIEW, 0, MetadataUtil.PENDING_NO_OP);
+            Dataset view = new Dataset(databaseName, dataverseName, viewName, itemTypeDatabaseName,
+                    itemTypeDataverseName, itemTypeName, MetadataConstants.METADATA_NODEGROUP_NAME, "",
+                    Collections.emptyMap(), viewDetails, Collections.emptyMap(), DatasetType.VIEW, 0,
+                    MetadataUtil.PENDING_NO_OP, creator);
             if (existingDataset == null) {
                 if (itemTypeIsInline) {
                     MetadataManager.INSTANCE.addDatatype(mdTxnCtx, itemTypeEntity);
@@ -2974,6 +3009,7 @@
                 }
                 MetadataManager.INSTANCE.updateDataset(mdTxnCtx, view);
             }
+            beforeTxnCommit(metadataProvider, creator, EntityDetails.newView(databaseName, dataverseName, viewName));
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             return existingDataset != null ? CreateResult.REPLACED : CreateResult.CREATED;
         } catch (Exception e) {
@@ -3065,7 +3101,7 @@
     }
 
     public void handleCreateFunctionStatement(MetadataProvider metadataProvider, Statement stmt,
-            IStatementRewriter stmtRewriter, IRequestParameters requestParameters) throws Exception {
+            IStatementRewriter stmtRewriter, IRequestParameters requestParameters, Creator creator) throws Exception {
         CreateFunctionStatement cfs = (CreateFunctionStatement) stmt;
         FunctionSignature signature = cfs.getFunctionSignature();
         DataverseName funDataverse = signature.getDataverseName();
@@ -3098,7 +3134,7 @@
         lockUtil.createFunctionBegin(lockManager, metadataProvider.getLocks(), databaseName, dataverseName,
                 signature.getName(), libraryDatabaseName, libraryDataverseName, libraryName);
         try {
-            doCreateFunction(metadataProvider, cfs, signature, stmtRewriter, requestParameters);
+            doCreateFunction(metadataProvider, cfs, signature, stmtRewriter, requestParameters, creator);
         } finally {
             metadataProvider.getLocks().unlock();
             metadataProvider.setDefaultNamespace(activeNamespace);
@@ -3106,8 +3142,8 @@
     }
 
     protected CreateResult doCreateFunction(MetadataProvider metadataProvider, CreateFunctionStatement cfs,
-            FunctionSignature functionSignature, IStatementRewriter stmtRewriter, IRequestParameters requestParameters)
-            throws Exception {
+            FunctionSignature functionSignature, IStatementRewriter stmtRewriter, IRequestParameters requestParameters,
+            Creator creator) throws Exception {
         DataverseName dataverseName = functionSignature.getDataverseName();
         String databaseName = functionSignature.getDatabaseName();
         SourceLocation sourceLoc = cfs.getSourceLocation();
@@ -3224,7 +3260,7 @@
                 function = new Function(functionSignature, paramNames, paramTypes, returnTypeSignature, null,
                         FunctionKind.SCALAR.toString(), library.getLanguage(), libraryDatabaseName,
                         libraryDataverseName, libraryName, externalIdentifier, cfs.getNullCall(),
-                        cfs.getDeterministic(), cfs.getResources(), dependencies);
+                        cfs.getDeterministic(), cfs.getResources(), dependencies, creator);
             } else {
                 List<Pair<VarIdentifier, TypeExpression>> paramList = cfs.getParameters();
                 int paramCount = paramList.size();
@@ -3263,7 +3299,7 @@
                 newInlineTypes = Collections.emptyMap();
                 function = new Function(functionSignature, paramNames, null, null, cfs.getFunctionBody(),
                         FunctionKind.SCALAR.toString(), compilationProvider.getParserFactory().getLanguage(), null,
-                        null, null, null, null, null, null, dependencies);
+                        null, null, null, null, null, null, dependencies, creator);
             }
 
             if (existingFunction == null) {
@@ -3289,6 +3325,8 @@
                 }
                 MetadataManager.INSTANCE.updateFunction(mdTxnCtx, function);
             }
+            beforeTxnCommit(metadataProvider, creator,
+                    EntityDetails.newFunction(databaseName, dataverseName, function.getName(), function.getArity()));
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             if (LOGGER.isInfoEnabled()) {
                 LOGGER.info("Installed function: " + functionSignature);
@@ -3809,7 +3847,7 @@
     }
 
     protected void handleCreateSynonymStatement(MetadataProvider metadataProvider, Statement stmt,
-            IRequestParameters requestParameters) throws Exception {
+            IRequestParameters requestParameters, Creator creator) throws Exception {
         CreateSynonymStatement css = (CreateSynonymStatement) stmt;
         metadataProvider.validateDatabaseObjectName(css.getNamespace(), css.getSynonymName(), css.getSourceLocation());
         Namespace stmtActiveNamespace = getActiveNamespace(css.getNamespace());
@@ -3823,14 +3861,16 @@
         }
         lockUtil.createSynonymBegin(lockManager, metadataProvider.getLocks(), databaseName, dataverseName, synonymName);
         try {
-            doCreateSynonym(metadataProvider, css, stmtActiveNamespace, synonymName, objectNamespace, objectName);
+            doCreateSynonym(metadataProvider, css, stmtActiveNamespace, synonymName, objectNamespace, objectName,
+                    creator);
         } finally {
             metadataProvider.getLocks().unlock();
         }
     }
 
     protected CreateResult doCreateSynonym(MetadataProvider metadataProvider, CreateSynonymStatement css,
-            Namespace namespace, String synonymName, Namespace objectNamespace, String objectName) throws Exception {
+            Namespace namespace, String synonymName, Namespace objectNamespace, String objectName, Creator creator)
+            throws Exception {
         String databaseName = namespace.getDatabaseName();
         DataverseName dataverseName = namespace.getDataverseName();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -3855,8 +3895,10 @@
                 throw new CompilationException(ErrorCode.SYNONYM_EXISTS, css.getSourceLocation(), synonymName);
             }
             synonym = new Synonym(databaseName, dataverseName, synonymName, objectNamespace.getDatabaseName(),
-                    objectNamespace.getDataverseName(), objectName);
+                    objectNamespace.getDataverseName(), objectName, creator);
             MetadataManager.INSTANCE.addSynonym(metadataProvider.getMetadataTxnContext(), synonym);
+            beforeTxnCommit(metadataProvider, creator,
+                    EntityDetails.newSynonym(databaseName, dataverseName, synonymName));
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             return CreateResult.CREATED;
         } catch (Exception e) {
@@ -4089,6 +4131,37 @@
                         ExternalDataConstants.WRITER_SUPPORTED_ADAPTERS, copyTo.getSourceLocation(), mdTxnCtx,
                         metadataProvider));
 
+                if (ExternalDataConstants.FORMAT_PARQUET
+                        .equalsIgnoreCase(edd.getProperties().get(ExternalDataConstants.KEY_FORMAT))) {
+                    if (copyTo.getType() != null) {
+                        DataverseName dataverseName =
+                                DataverseName.createFromCanonicalForm(ExternalDataConstants.DUMMY_DATAVERSE_NAME);
+                        IAType iaType = translateType(ExternalDataConstants.DUMMY_DATABASE_NAME, dataverseName,
+                                ExternalDataConstants.DUMMY_TYPE_NAME, copyTo.getType(), mdTxnCtx);
+                        edd.getProperties().put(ExternalDataConstants.PARQUET_SCHEMA_KEY,
+                                SchemaConverterVisitor.convertToParquetSchemaString((ARecordType) iaType));
+                    }
+                }
+
+                if (edd.getProperties().get(ExternalDataConstants.KEY_FORMAT)
+                        .equalsIgnoreCase(ExternalDataConstants.FORMAT_CSV_LOWER_CASE)) {
+                    DataverseName dataverseName =
+                            DataverseName.createFromCanonicalForm(ExternalDataConstants.DUMMY_DATAVERSE_NAME);
+                    IAType iaType;
+                    if (copyTo.getType() != null) {
+                        iaType = translateType(ExternalDataConstants.DUMMY_DATABASE_NAME, dataverseName,
+                                ExternalDataConstants.DUMMY_TYPE_NAME, copyTo.getType(), mdTxnCtx);
+                    } else if (copyTo.getTypeExpressionItemType() != null) {
+                        iaType = translateType(ExternalDataConstants.DUMMY_DATABASE_NAME, dataverseName,
+                                ExternalDataConstants.DUMMY_TYPE_NAME, copyTo.getTypeExpressionItemType(), mdTxnCtx);
+                    } else {
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR,
+                                "TYPE/AS Expression is required for csv format");
+                    }
+                    ARecordType recordType = (ARecordType) iaType;
+                    validateCSVSchema(recordType);
+                    edd.setItemType(recordType);
+                }
                 Map<VarIdentifier, IAObject> externalVars = createExternalVariables(copyTo, stmtParams);
                 // Query Rewriting (happens under the same ongoing metadata transaction)
                 LangRewritingContext langRewritingContext = createLangRewritingContext(metadataProvider,
@@ -4851,7 +4924,7 @@
                     dsDetails.getKeySourceIndicator(), dsDetails.getPrimaryKeyType(), sampleCardinalityTarget, 0, 0,
                     sampleSeed, Collections.emptyMap());
             newIndexPendingAdd = new Index(databaseName, dataverseName, datasetName, newIndexName, sampleIndexType,
-                    newIndexDetailsPendingAdd, false, false, MetadataUtil.PENDING_ADD_OP);
+                    newIndexDetailsPendingAdd, false, false, MetadataUtil.PENDING_ADD_OP, Creator.DEFAULT_CREATOR);
 
             // #. add a new index with PendingAddOp
             MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), newIndexPendingAdd);
@@ -4893,7 +4966,7 @@
                     dsDetails.getKeySourceIndicator(), dsDetails.getPrimaryKeyType(), sampleCardinalityTarget,
                     stats.getCardinality(), stats.getAvgTupleSize(), sampleSeed, stats.getIndexesStats());
             Index newIndexFinal = new Index(databaseName, dataverseName, datasetName, newIndexName, sampleIndexType,
-                    newIndexDetailsFinal, false, false, MetadataUtil.PENDING_NO_OP);
+                    newIndexDetailsFinal, false, false, MetadataUtil.PENDING_NO_OP, Creator.DEFAULT_CREATOR);
 
             // #. begin new metadataTxn
             mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -5157,7 +5230,7 @@
             MetadataManager.INSTANCE.addIndex(mdTxnCtx,
                     new Index(databaseName, dataverseName, datasetName, indexName, index.getIndexType(),
                             index.getIndexDetails(), index.isEnforced(), index.isPrimaryIndex(),
-                            MetadataUtil.PENDING_DROP_OP));
+                            MetadataUtil.PENDING_DROP_OP, index.getCreator()));
         }
     }
 
@@ -5456,6 +5529,7 @@
                             participatingDatasetIds, numParticipatingNodes, numParticipatingPartitions));
                 }
             }
+
             jobId = runTrackJob(hcc, jobSpec, jobFlags, reqId, requestParameters.getClientContextId(), clientRequest);
             if (jId != null) {
                 jId.setValue(jobId);
@@ -5694,11 +5768,28 @@
         }
     }
 
+    /**
+     * Normalizes the value of the adapter and ensures that it is supported
+     *
+     * @param details external details
+     * @param sourceLoc source location
+     */
+    private void normalizeAdapters(ExternalDetailsDecl details, SourceLocation sourceLoc) throws CompilationException {
+        String adapter = details.getAdapter();
+        Optional<String> normalizedAdapter = ExternalDataConstants.EXTERNAL_READ_ADAPTERS.stream()
+                .filter(k -> k.equalsIgnoreCase(adapter)).findFirst();
+        if (normalizedAdapter.isEmpty()) {
+            throw CompilationException.create(ErrorCode.UNKNOWN_ADAPTER, sourceLoc, adapter);
+        }
+        details.setAdapter(normalizedAdapter.get());
+    }
+
     protected void validateExternalDatasetProperties(ExternalDetailsDecl externalDetails,
             Map<String, String> properties, SourceLocation srcLoc, MetadataTransactionContext mdTxnCtx,
             IApplicationContext appCtx, MetadataProvider metadataProvider)
             throws AlgebricksException, HyracksDataException {
         // Validate adapter specific properties
+        normalizeAdapters(externalDetails, srcLoc);
         String adapter = externalDetails.getAdapter();
         Map<String, String> details = new HashMap<>(properties);
         details.put(ExternalDataConstants.KEY_EXTERNAL_SOURCE_TYPE, adapter);
@@ -5800,10 +5891,35 @@
         }
     }
 
+    protected void beforeTxnCommit(MetadataProvider metadataProvider, Creator creator, EntityDetails entityDetails)
+            throws AlgebricksException {
+        //no op
+    }
+
     protected enum CreateResult {
         NOOP,
         CREATED,
         REPLACED
     }
 
+    private void validateCSVSchema(ARecordType schema) throws CompilationException {
+        final List<String> expectedFieldNames = Arrays.asList(schema.getFieldNames());
+        final List<IAType> expectedFieldTypes = Arrays.asList(schema.getFieldTypes());
+        final int size = expectedFieldNames.size();
+        for (int i = 0; i < size; ++i) {
+            IAType expectedIAType = expectedFieldTypes.get(i);
+            if (!ExternalDataConstants.CSV_WRITER_SUPPORTED_DATA_TYPES.contains(expectedIAType.getTypeTag())) {
+                if (expectedIAType.getTypeTag().equals(ATypeTag.UNION)) {
+                    AUnionType unionType = (AUnionType) expectedIAType;
+                    ATypeTag actualTypeTag = unionType.getActualType().getTypeTag();
+                    if (!ExternalDataConstants.CSV_WRITER_SUPPORTED_DATA_TYPES.contains(actualTypeTag)) {
+                        throw new CompilationException(ErrorCode.TYPE_UNSUPPORTED_CSV_WRITE, actualTypeTag.toString());
+                    }
+                } else {
+                    throw new CompilationException(ErrorCode.TYPE_UNSUPPORTED_CSV_WRITE,
+                            expectedIAType.getTypeTag().toString());
+                }
+            }
+        }
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
index 0b7e7d0..6928b64 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
@@ -91,6 +91,7 @@
 import org.apache.asterix.common.metadata.NamespacePathResolver;
 import org.apache.asterix.common.metadata.NamespaceResolver;
 import org.apache.asterix.common.replication.INcLifecycleCoordinator;
+import org.apache.asterix.common.utils.IdentifierUtil;
 import org.apache.asterix.common.utils.Servlets;
 import org.apache.asterix.external.adapter.factory.AdapterFactoryService;
 import org.apache.asterix.file.StorageComponentProvider;
@@ -165,7 +166,7 @@
         ccServiceCtx.setMessageBroker(new CCMessageBroker(controllerService));
         ccServiceCtx.setPersistedResourceRegistry(new PersistedResourceRegistry());
         configureLoggingLevel(ccServiceCtx.getAppConfig().getLoggingLevel(ExternalProperties.Option.LOG_LEVEL));
-        LOGGER.info("Starting Asterix cluster controller");
+        LOGGER.info("Starting {} cluster controller", IdentifierUtil.productName());
         String strIP = ccServiceCtx.getCCContext().getClusterControllerInfo().getClientNetAddress();
         int port = ccServiceCtx.getCCContext().getClusterControllerInfo().getClientNetPort();
         hcc = new HyracksConnection(strIP, port,
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
index 0ff8796..6f8126b 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
@@ -79,6 +79,7 @@
 import org.apache.asterix.common.transactions.IRecoveryManager;
 import org.apache.asterix.common.transactions.IRecoveryManager.SystemState;
 import org.apache.asterix.common.transactions.IRecoveryManagerFactory;
+import org.apache.asterix.common.utils.IdentifierUtil;
 import org.apache.asterix.common.utils.PrintUtil;
 import org.apache.asterix.common.utils.Servlets;
 import org.apache.asterix.common.utils.StorageConstants;
@@ -148,9 +149,7 @@
             throw new IllegalArgumentException("Unrecognized argument(s): " + Arrays.toString(args));
         }
         nodeId = this.ncServiceCtx.getNodeId();
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Starting Asterix node controller: " + nodeId);
-        }
+        LOGGER.info("Starting {} node controller: {}", IdentifierUtil.productName(), nodeId);
         final NodeControllerService controllerService = (NodeControllerService) ncServiceCtx.getControllerService();
 
         if (System.getProperty("java.rmi.server.hostname") == null) {
@@ -280,9 +279,7 @@
         if (!stopInitiated) {
             runtimeContext.setShuttingdown(true);
             stopInitiated = true;
-            if (LOGGER.isInfoEnabled()) {
-                LOGGER.info("Stopping Asterix node controller: " + nodeId);
-            }
+            LOGGER.info("Stopping {} node controller: {}", IdentifierUtil.productName(), nodeId);
 
             webManager.stop();
 
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java
index 9d39088..ea8d5fc 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java
@@ -31,7 +31,9 @@
 import org.apache.asterix.app.function.StorageComponentsRewriter;
 import org.apache.asterix.app.function.TPCDSAllTablesDataGeneratorRewriter;
 import org.apache.asterix.app.function.TPCDSSingleTableDataGeneratorRewriter;
+import org.apache.asterix.app.function.collectionsize.StorageSizeRewriter;
 import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.typecomputer.impl.AInt64TypeComputer;
 import org.apache.asterix.om.utils.RecordUtil;
 
 public class MetadataBuiltinFunctions {
@@ -106,6 +108,10 @@
                 true);
         BuiltinFunctions.addUnnestFun(QueryPartitionRewriter.QUERY_PARTITION, false);
         BuiltinFunctions.addDatasourceFunction(QueryPartitionRewriter.QUERY_PARTITION, QueryPartitionRewriter.INSTANCE);
+        // storage size
+        BuiltinFunctions.addFunction(StorageSizeRewriter.STORAGE_SIZE, AInt64TypeComputer.INSTANCE, true);
+        BuiltinFunctions.addUnnestFun(StorageSizeRewriter.STORAGE_SIZE, true);
+        BuiltinFunctions.addDatasourceFunction(StorageSizeRewriter.STORAGE_SIZE, StorageSizeRewriter.INSTANCE);
     }
 
     private MetadataBuiltinFunctions() {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/StorageUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/StorageUtil.java
new file mode 100644
index 0000000..3eb9d08
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/StorageUtil.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.utils;
+
+import static org.apache.asterix.common.api.IClusterManagementWork.ClusterState.ACTIVE;
+import static org.apache.asterix.common.api.IClusterManagementWork.ClusterState.REBALANCE_REQUIRED;
+import static org.apache.asterix.common.exceptions.ErrorCode.REJECT_BAD_CLUSTER_STATE;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.asterix.app.message.StorageSizeRequestMessage;
+import org.apache.asterix.common.api.IClusterManagementWork;
+import org.apache.asterix.common.api.IMetadataLockManager;
+import org.apache.asterix.common.config.DatasetConfig;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.common.metadata.MetadataConstants;
+import org.apache.asterix.common.metadata.MetadataUtil;
+import org.apache.asterix.messaging.CCMessageBroker;
+import org.apache.asterix.metadata.MetadataManager;
+import org.apache.asterix.metadata.MetadataTransactionContext;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.hyracks.api.util.InvokeUtil;
+
+public class StorageUtil {
+
+    public static long getCollectionSize(ICcApplicationContext appCtx, String database, DataverseName dataverse,
+            String collection, String index) throws Exception {
+        IClusterManagementWork.ClusterState state = appCtx.getClusterStateManager().getState();
+        if (!(state == ACTIVE || state == REBALANCE_REQUIRED)) {
+            throw new RuntimeDataException(REJECT_BAD_CLUSTER_STATE, state);
+        }
+
+        if (!appCtx.getNamespaceResolver().isUsingDatabase()) {
+            database = MetadataConstants.DEFAULT_DATABASE;
+        }
+
+        IMetadataLockManager lockManager = appCtx.getMetadataLockManager();
+        MetadataProvider metadataProvider = MetadataProvider.createWithDefaultNamespace(appCtx);
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        try {
+            lockManager.acquireDatabaseReadLock(metadataProvider.getLocks(), database);
+            lockManager.acquireDataverseReadLock(metadataProvider.getLocks(), database, dataverse);
+            lockManager.acquireDatasetReadLock(metadataProvider.getLocks(), database, dataverse, collection);
+            Dataset dataset = metadataProvider.findDataset(database, dataverse, collection);
+            if (dataset == null) {
+                throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, collection,
+                        MetadataUtil.dataverseName(database, dataverse, metadataProvider.isUsingDatabase()));
+            }
+
+            if (dataset.getDatasetType() != DatasetConfig.DatasetType.INTERNAL) {
+                throw new CompilationException(ErrorCode.STORAGE_SIZE_NOT_APPLICABLE_TO_TYPE, dataset.getDatasetType());
+            }
+
+            if (index != null) {
+                Index idx = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(), database,
+                        dataverse, collection, index);
+                if (idx == null) {
+                    throw new CompilationException(ErrorCode.UNKNOWN_INDEX, index);
+                }
+            }
+
+            final List<String> ncs = new ArrayList<>(appCtx.getClusterStateManager().getParticipantNodes());
+            CCMessageBroker messageBroker = (CCMessageBroker) appCtx.getServiceContext().getMessageBroker();
+
+            long reqId = messageBroker.newRequestId();
+            List<StorageSizeRequestMessage> requests = new ArrayList<>();
+            for (int i = 0; i < ncs.size(); i++) {
+                requests.add(new StorageSizeRequestMessage(reqId, database, dataverse.getCanonicalForm(), collection,
+                        index));
+            }
+            return (long) messageBroker.sendSyncRequestToNCs(reqId, ncs, requests, TimeUnit.SECONDS.toMillis(60), true);
+        } finally {
+            InvokeUtil.tryWithCleanups(() -> MetadataManager.INSTANCE.commitTransaction(mdTxnCtx),
+                    () -> metadataProvider.getLocks().unlock());
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/resources/cc.conf b/asterixdb/asterix-app/src/main/resources/cc.conf
index e0eb88d..f5a496c 100644
--- a/asterixdb/asterix-app/src/main/resources/cc.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc.conf
@@ -59,6 +59,7 @@
 compiler.windowmemory=192KB
 compiler.sort.parallel=false
 compiler.internal.sanitycheck=true
+compiler.ordered.fields=false
 messaging.frame.size=4096
 messaging.frame.count=512
 storage.buffercache.pagesize=32KB
diff --git a/asterixdb/asterix-app/src/main/resources/cc2.conf b/asterixdb/asterix-app/src/main/resources/cc2.conf
index 8868a54..bb2e457 100644
--- a/asterixdb/asterix-app/src/main/resources/cc2.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc2.conf
@@ -53,6 +53,7 @@
 compiler.joinmemory=256KB
 compiler.textsearchmemory=160KB
 compiler.windowmemory=192KB
+compiler.ordered.fields=false
 compiler.parallelism=-1
 messaging.frame.size=4096
 messaging.frame.count=512
diff --git a/asterixdb/asterix-app/src/main/resources/cc3.conf b/asterixdb/asterix-app/src/main/resources/cc3.conf
index 71ad17b..cc83da0 100644
--- a/asterixdb/asterix-app/src/main/resources/cc3.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc3.conf
@@ -53,6 +53,7 @@
 compiler.joinmemory=256KB
 compiler.textsearchmemory=160KB
 compiler.windowmemory=192KB
+compiler.ordered.fields=false
 compiler.parallelism=3
 messaging.frame.size=4096
 messaging.frame.count=512
diff --git a/asterixdb/asterix-app/src/main/resources/cc4.conf b/asterixdb/asterix-app/src/main/resources/cc4.conf
index bf37499..0d5b9ad 100644
--- a/asterixdb/asterix-app/src/main/resources/cc4.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc4.conf
@@ -50,6 +50,7 @@
 compiler.groupmemory=160KB
 compiler.joinmemory=256KB
 compiler.windowmemory=192KB
+compiler.ordered.fields=false
 messaging.frame.size=4096
 messaging.frame.count=512
 compiler.parallelism=-1
diff --git a/asterixdb/asterix-app/src/main/resources/cc_no_cbo.conf b/asterixdb/asterix-app/src/main/resources/cc_no_cbo.conf
index ab028c8..6b497e2 100644
--- a/asterixdb/asterix-app/src/main/resources/cc_no_cbo.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc_no_cbo.conf
@@ -60,6 +60,7 @@
 compiler.textsearchmemory=160KB
 compiler.windowmemory=192KB
 compiler.sort.parallel=false
+compiler.ordered.fields=false
 compiler.internal.sanitycheck=true
 messaging.frame.size=4096
 messaging.frame.count=512
diff --git a/asterixdb/asterix-app/src/main/resources/cc_static_partitioning.conf b/asterixdb/asterix-app/src/main/resources/cc_static_partitioning.conf
index 9d28e55..26db39f 100644
--- a/asterixdb/asterix-app/src/main/resources/cc_static_partitioning.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc_static_partitioning.conf
@@ -58,6 +58,7 @@
 compiler.textsearchmemory=160KB
 compiler.windowmemory=192KB
 compiler.sort.parallel=false
+compiler.ordered.fields=false
 compiler.internal.sanitycheck=true
 messaging.frame.size=4096
 messaging.frame.count=512
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
index e312b01..54b4004 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
@@ -74,6 +74,7 @@
     public static final int DEFAULT_HYRACKS_CC_CLUSTER_PORT = 1099;
     public static final String RESOURCES_PATH = joinPath(getProjectPath().toString(), "src", "test", "resources");
     public static final String DEFAULT_CONF_FILE = joinPath(RESOURCES_PATH, "cc.conf");
+    public static final String DEFAULT_MAIN_CONF_FILE = joinPath(RESOURCES_PATH, "cc-main.conf");
     private static final String DEFAULT_STORAGE_PATH = joinPath("target", "io", "dir");
     private static String storagePath = DEFAULT_STORAGE_PATH;
     private static final long RESULT_TTL = TimeUnit.MINUTES.toMillis(30);
@@ -469,7 +470,7 @@
     private static String getConfPath() {
         String providedPath = System.getProperty("conf.path");
         if (providedPath == null) {
-            return DEFAULT_CONF_FILE;
+            return DEFAULT_MAIN_CONF_FILE;
         }
         return joinPath(RESOURCES_PATH, providedPath);
     }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/CloudStorageIntegrationUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/CloudStorageIntegrationUtil.java
index 47c0599..6ee625f 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/CloudStorageIntegrationUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/CloudStorageIntegrationUtil.java
@@ -24,7 +24,7 @@
 public class CloudStorageIntegrationUtil extends AsterixHyracksIntegrationUtil {
 
     public static final String RESOURCES_PATH = joinPath(getProjectPath().toString(), "src", "test", "resources");
-    public static final String CONFIG_FILE = joinPath(RESOURCES_PATH, "cc-cloud-storage.conf");
+    public static final String CONFIG_FILE = joinPath(RESOURCES_PATH, "cc-cloud-storage-main.conf");
 
     public static void main(String[] args) throws Exception {
         boolean cleanStart = Boolean.getBoolean("cleanup.start");
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/LocalCloudUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/LocalCloudUtil.java
index 9e3fdc4..b50d2f2 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/LocalCloudUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/LocalCloudUtil.java
@@ -57,6 +57,10 @@
     }
 
     public static S3Mock startS3CloudEnvironment(boolean cleanStart) {
+        return startS3CloudEnvironment(cleanStart, false);
+    }
+
+    public static S3Mock startS3CloudEnvironment(boolean cleanStart, boolean createPlaygroundContainer) {
         if (cleanStart) {
             FileUtils.deleteQuietly(new File(MOCK_FILE_BACKEND));
         }
@@ -79,6 +83,12 @@
         S3Client client = builder.build();
         client.createBucket(CreateBucketRequest.builder().bucket(CLOUD_STORAGE_BUCKET).build());
         LOGGER.info("Created bucket {} for cloud storage", CLOUD_STORAGE_BUCKET);
+
+        // added for convenience since some non-external-based tests include an external collection test on this bucket
+        if (createPlaygroundContainer) {
+            client.createBucket(CreateBucketRequest.builder().bucket("playground").build());
+            LOGGER.info("Created bucket {}", "playground");
+        }
         client.close();
         return s3MockServer;
     }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
index 8aa25ad..aaee17e 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
@@ -53,6 +53,7 @@
 import org.apache.asterix.metadata.entities.Dataverse;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.NoOpLSMTupleFilterCallbackFactory;
+import org.apache.asterix.metadata.utils.Creator;
 import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.metadata.utils.SplitsAndConstraintsUtil;
 import org.apache.asterix.om.types.ARecordType;
@@ -500,7 +501,7 @@
         PrimaryIndexInfo primaryIndexInfo = new PrimaryIndexInfo(dataset, primaryKeyTypes, recordType, metaType,
                 mergePolicy.first, mergePolicy.second, filterFields, primaryKeyIndexes, primaryKeyIndicators);
         Dataverse dataverse = new Dataverse(dataset.getDatabaseName(), dataset.getDataverseName(),
-                NonTaggedDataFormat.class.getName(), MetadataUtil.PENDING_NO_OP);
+                NonTaggedDataFormat.class.getName(), MetadataUtil.PENDING_NO_OP, Creator.DEFAULT_CREATOR);
         Namespace namespace = new Namespace(dataverse.getDatabaseName(), dataverse.getDataverseName());
         MetadataProvider mdProvider = MetadataProvider.create(
                 (ICcApplicationContext) ExecutionTestUtil.integrationUtil.cc.getApplicationContext(), namespace);
@@ -528,7 +529,7 @@
         MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
         Dataverse dataverse =
                 new Dataverse(primaryIndexInfo.dataset.getDatabaseName(), primaryIndexInfo.dataset.getDataverseName(),
-                        NonTaggedDataFormat.class.getName(), MetadataUtil.PENDING_NO_OP);
+                        NonTaggedDataFormat.class.getName(), MetadataUtil.PENDING_NO_OP, Creator.DEFAULT_CREATOR);
         Namespace namespace = new Namespace(dataverse.getDatabaseName(), dataverse.getDataverseName());
         MetadataProvider mdProvider = MetadataProvider.create(
                 (ICcApplicationContext) ExecutionTestUtil.integrationUtil.cc.getApplicationContext(), namespace);
@@ -746,7 +747,7 @@
             }
             index = Index.createPrimaryIndex(dataset.getDatabaseName(), dataset.getDataverseName(),
                     dataset.getDatasetName(), keyFieldNames, primaryKeyIndicators, keyFieldTypes,
-                    MetadataUtil.PENDING_NO_OP);
+                    MetadataUtil.PENDING_NO_OP, dataset.getCreator());
             List<String> nodes = Collections.singletonList(ExecutionTestUtil.integrationUtil.ncs[0].getId());
             CcApplicationContext appCtx =
                     (CcApplicationContext) ExecutionTestUtil.integrationUtil.cc.getApplicationContext();
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
index 841e81b..460565c 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
@@ -114,9 +114,9 @@
         handler = new ActiveNotificationHandler();
         allDatasets = new ArrayList<>();
         firstDataset = new Dataset(database, dataverseName, "firstDataset", recordTypeDatabaseName, null, null, null,
-                null, null, null, null, null, 0, 0);
+                null, null, null, null, null, 0, 0, null);
         secondDataset = new Dataset(database, dataverseName, "secondDataset", recordTypeDatabaseName, null, null, null,
-                null, null, null, null, null, 0, 0);
+                null, null, null, null, null, 0, 0, null);
         allDatasets.add(firstDataset);
         allDatasets.add(secondDataset);
         AtomicInteger threadCounter = new AtomicInteger(0);
@@ -992,7 +992,7 @@
                 new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RECOVERING));
         recoveringSubscriber.sync();
         Dataset newDataset = new Dataset(database, dataverseName, "newDataset", recordTypeDatabaseName, null, null,
-                null, null, null, null, null, null, 0, 0);
+                null, null, null, null, null, null, 0, 0, null);
         Action add = users[1].addDataset(newDataset, listener);
         listener.allowStep();
         runningSubscriber.sync();
@@ -1019,7 +1019,7 @@
         recoveringSubscriber.sync();
         tempFailSubscriber = new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
         Dataset newDataset = new Dataset(database, dataverseName, "newDataset", recordTypeDatabaseName, null, null,
-                null, null, null, null, null, null, 0, 0);
+                null, null, null, null, null, null, 0, 0, null);
         Action add = users[1].addDataset(newDataset, listener);
         listener.allowStep();
         tempFailSubscriber.sync();
@@ -1046,7 +1046,7 @@
         recoveringSubscriber.sync();
         tempFailSubscriber = new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
         Dataset newDataset = new Dataset(database, dataverseName, "newDataset", recordTypeDatabaseName, null, null,
-                null, null, null, null, null, null, 0, 0);
+                null, null, null, null, null, null, 0, 0, null);
         Action add = users[1].addDataset(newDataset, listener);
         listener.allowStep();
         tempFailSubscriber.sync();
@@ -1065,7 +1065,7 @@
                 new WaitForStateSubscriber(listener, Collections.singleton(ActivityState.STARTING));
         subscriber.sync();
         Dataset newDataset = new Dataset(database, dataverseName, "newDataset", recordTypeDatabaseName, null, null,
-                null, null, null, null, null, null, 0, 0);
+                null, null, null, null, null, null, 0, 0, null);
         Action createDatasetAction = users[1].addDataset(newDataset, listener);
         listener.allowStep();
         startAction.sync();
@@ -1081,7 +1081,7 @@
     public void testCreateNewDatasetWhileRunning() throws Exception {
         testStartWhenStartSucceed();
         Dataset newDataset = new Dataset(database, dataverseName, "newDataset", recordTypeDatabaseName, null, null,
-                null, null, null, null, null, null, 0, 0);
+                null, null, null, null, null, null, 0, 0, null);
         Action createDatasetAction = users[1].addDataset(newDataset, listener);
         createDatasetAction.sync();
         assertFailure(createDatasetAction, ErrorCode.CANNOT_ADD_DATASET_TO_ACTIVE_ENTITY);
@@ -1101,7 +1101,7 @@
                 new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.SUSPENDING, ActivityState.SUSPENDED));
         subscriber.sync();
         Dataset newDataset = new Dataset(database, dataverseName, "newDataset", recordTypeDatabaseName, null, null,
-                null, null, null, null, null, null, 0, 0);
+                null, null, null, null, null, null, 0, 0, null);
         Action createDatasetAction = users[0].addDataset(newDataset, listener);
         listener.allowStep();
         listener.allowStep();
@@ -1120,7 +1120,7 @@
         testRecoveryFailureAfterOneAttemptCompilationFailure();
         Assert.assertEquals(ActivityState.STOPPED, listener.getState());
         Dataset newDataset = new Dataset(database, dataverseName, "newDataset", recordTypeDatabaseName, null, null,
-                null, null, null, null, null, null, 0, 0);
+                null, null, null, null, null, null, 0, 0, null);
         Action createDatasetAction = users[0].addDataset(newDataset, listener);
         createDatasetAction.sync();
         assertSuccess(createDatasetAction);
@@ -1553,7 +1553,7 @@
         query.sync();
         assertSuccess(query);
         Dataset newDataset = new Dataset(database, dataverseName, "newDataset", recordTypeDatabaseName, null, null,
-                null, null, null, null, null, null, 0, 0);
+                null, null, null, null, null, null, 0, 0, null);
         Action addDataset = users[1].addDataset(newDataset, listener);
         // blocked by suspension
         Assert.assertFalse(addDataset.isDone());
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageAzTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageAzTest.java
new file mode 100644
index 0000000..508810d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageAzTest.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.cloud_storage;
+
+import static org.apache.asterix.api.common.LocalCloudUtil.CLOUD_STORAGE_BUCKET;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Random;
+
+import org.apache.asterix.common.config.GlobalConfig;
+import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.test.runtime.LangExecutionUtil;
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.asterix.testframework.xml.Description;
+import org.apache.asterix.testframework.xml.TestCase;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.junit.AfterClass;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.FixMethodOrder;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.MethodSorters;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.BlobServiceClientBuilder;
+import com.azure.storage.common.StorageSharedKeyCredential;
+
+/**
+ * Run tests in cloud deployment environment
+ */
+@RunWith(Parameterized.class)
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+public class CloudStorageAzTest {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+
+    private final TestCaseContext tcCtx;
+    private static final String SUITE_TESTS = "testsuite_cloud_storage.xml";
+    private static final String ONLY_TESTS = "testsuite_cloud_storage_only.xml";
+    private static final String CONFIG_FILE_NAME = "src/test/resources/cc-cloud-storage-azblob.conf";
+    private static final String DELTA_RESULT_PATH = "results_cloud";
+    private static final String EXCLUDED_TESTS = "MP";
+
+    public CloudStorageAzTest(TestCaseContext tcCtx) {
+        this.tcCtx = tcCtx;
+    }
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        String endpointString = "http://127.0.0.1:15055/devstoreaccount1/" + CLOUD_STORAGE_BUCKET;
+        final String accKey =
+                "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==";
+        final String accName = "devstoreaccount1";
+
+        BlobServiceClient blobServiceClient = new BlobServiceClientBuilder().endpoint(endpointString)
+                .credential(new StorageSharedKeyCredential(accName, accKey)).buildClient();
+
+        cleanup(blobServiceClient);
+        initialize(blobServiceClient);
+
+        //storage.close(); WHAT IS THIS FOR IN GCS
+
+        TestExecutor testExecutor = new TestExecutor(DELTA_RESULT_PATH);
+        testExecutor.executorId = "cloud";
+        testExecutor.stripSubstring = "//DB:";
+        LangExecutionUtil.setUp(CONFIG_FILE_NAME, testExecutor);
+        System.setProperty(GlobalConfig.CONFIG_FILE_PROPERTY, CONFIG_FILE_NAME);
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        LangExecutionUtil.tearDown();
+    }
+
+    @Parameters(name = "CloudStorageAzBlobTest {index}: {0}")
+    public static Collection<Object[]> tests() throws Exception {
+        long seed = System.nanoTime();
+        Random random = new Random(seed);
+        LOGGER.info("CloudStorageAzBlobTest seed {}", seed);
+        Collection<Object[]> tests = LangExecutionUtil.tests(ONLY_TESTS, SUITE_TESTS);
+        List<Object[]> selected = new ArrayList<>();
+        for (Object[] test : tests) {
+            if (!Objects.equals(((TestCaseContext) test[0]).getTestGroups()[0].getName(), "sqlpp_queries")) {
+                selected.add(test);
+            }
+            // Select 10% of the tests randomly
+            else if (random.nextInt(10) == 0) {
+                selected.add(test);
+            }
+        }
+        return selected;
+    }
+
+    @Test
+    public void test() throws Exception {
+        List<TestCase.CompilationUnit> cu = tcCtx.getTestCase().getCompilationUnit();
+        Assume.assumeTrue(cu.size() > 1 || !EXCLUDED_TESTS.equals(getText(cu.get(0).getDescription())));
+        LangExecutionUtil.test(tcCtx);
+    }
+
+    private static String getText(Description description) {
+        return description == null ? "" : description.getValue();
+    }
+
+    private static void cleanup(BlobServiceClient blobServiceClient) {
+        blobServiceClient.deleteBlobContainerIfExists(CLOUD_STORAGE_BUCKET);
+    }
+
+    private static void initialize(BlobServiceClient blobServiceClient) {
+        blobServiceClient.createBlobContainerIfNotExists(CLOUD_STORAGE_BUCKET);
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageGCSTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageGCSTest.java
index a60acb7..6ac4a5d 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageGCSTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageGCSTest.java
@@ -21,9 +21,13 @@
 import static org.apache.asterix.api.common.LocalCloudUtil.CLOUD_STORAGE_BUCKET;
 import static org.apache.asterix.api.common.LocalCloudUtil.MOCK_SERVER_REGION;
 
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
+import java.util.Objects;
+import java.util.Random;
 
+import org.apache.asterix.api.common.LocalCloudUtil;
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.test.common.TestExecutor;
 import org.apache.asterix.test.runtime.LangExecutionUtil;
@@ -36,7 +40,6 @@
 import org.junit.Assume;
 import org.junit.BeforeClass;
 import org.junit.FixMethodOrder;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.MethodSorters;
@@ -55,7 +58,6 @@
  */
 @RunWith(Parameterized.class)
 @FixMethodOrder(MethodSorters.NAME_ASCENDING)
-@Ignore
 public class CloudStorageGCSTest {
 
     private static final Logger LOGGER = LogManager.getLogger();
@@ -75,6 +77,7 @@
 
     @BeforeClass
     public static void setUp() throws Exception {
+        LocalCloudUtil.startS3CloudEnvironment(true, true);
         Storage storage = StorageOptions.newBuilder().setHost(MOCK_SERVER_HOSTNAME)
                 .setCredentials(NoCredentials.getInstance()).setProjectId(MOCK_SERVER_PROJECT_ID).build().getService();
         cleanup(storage);
@@ -94,7 +97,21 @@
 
     @Parameters(name = "CloudStorageGCSTest {index}: {0}")
     public static Collection<Object[]> tests() throws Exception {
-        return LangExecutionUtil.tests(ONLY_TESTS, SUITE_TESTS);
+        long seed = System.nanoTime();
+        Random random = new Random(seed);
+        LOGGER.info("CloudStorageGCSTest seed {}", seed);
+        Collection<Object[]> tests = LangExecutionUtil.tests(ONLY_TESTS, SUITE_TESTS);
+        List<Object[]> selected = new ArrayList<>();
+        for (Object[] test : tests) {
+            if (!Objects.equals(((TestCaseContext) test[0]).getTestGroups()[0].getName(), "sqlpp_queries")) {
+                selected.add(test);
+            }
+            // Select 10% of the tests randomly
+            else if (random.nextInt(10) == 0) {
+                selected.add(test);
+            }
+        }
+        return selected;
     }
 
     @Test
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageTest.java
index 5f7a037..78f4e55 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageTest.java
@@ -18,6 +18,7 @@
  */
 package org.apache.asterix.test.cloud_storage;
 
+import java.net.URI;
 import java.util.Collection;
 import java.util.List;
 
@@ -40,6 +41,12 @@
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
+import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.awssdk.services.s3.S3ClientBuilder;
+import software.amazon.awssdk.services.s3.model.CreateBucketRequest;
+
 /**
  * Run tests in cloud deployment environment
  */
@@ -56,6 +63,11 @@
     private static final String DELTA_RESULT_PATH = "results_cloud";
     private static final String EXCLUDED_TESTS = "MP";
 
+    private static final String PLAYGROUND_CONTAINER = "playground";
+    private static final String MOCK_SERVER_REGION = "us-west-2";
+    private static final int MOCK_SERVER_PORT = 8001;
+    private static final String MOCK_SERVER_HOSTNAME = "http://127.0.0.1:" + MOCK_SERVER_PORT;
+
     public CloudStorageTest(TestCaseContext tcCtx) {
         this.tcCtx = tcCtx;
     }
@@ -68,6 +80,15 @@
         testExecutor.stripSubstring = "//DB:";
         LangExecutionUtil.setUp(CONFIG_FILE_NAME, testExecutor);
         System.setProperty(GlobalConfig.CONFIG_FILE_PROPERTY, CONFIG_FILE_NAME);
+
+        // create the playground bucket and leave it empty, just for external collection-based tests
+        S3ClientBuilder builder = S3Client.builder();
+        URI endpoint = URI.create(MOCK_SERVER_HOSTNAME); // endpoint pointing to S3 mock server
+        builder.region(Region.of(MOCK_SERVER_REGION)).credentialsProvider(AnonymousCredentialsProvider.create())
+                .endpointOverride(endpoint);
+        S3Client client = builder.build();
+        client.createBucket(CreateBucketRequest.builder().bucket(PLAYGROUND_CONTAINER).build());
+        client.close();
     }
 
     @AfterClass
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageUnstableTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageUnstableTest.java
index a517ed4..82a264a 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageUnstableTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/cloud_storage/CloudStorageUnstableTest.java
@@ -66,7 +66,7 @@
     @BeforeClass
     public static void setUp() throws Exception {
         System.setProperty(CloudRetryableRequestUtil.CLOUD_UNSTABLE_MODE, "true");
-        LocalCloudUtil.startS3CloudEnvironment(true);
+        LocalCloudUtil.startS3CloudEnvironment(true, true);
         TestExecutor testExecutor = new TestExecutor(DELTA_RESULT_PATH);
         testExecutor.executorId = "cloud";
         testExecutor.stripSubstring = "//DB:";
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/AnalyzingTestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/AnalyzingTestExecutor.java
index 2658804..24191d8 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/AnalyzingTestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/AnalyzingTestExecutor.java
@@ -25,6 +25,7 @@
 import java.util.regex.Pattern;
 
 import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.asterix.testframework.xml.TestCase;
 import org.apache.commons.io.IOUtils;
 
 import com.fasterxml.jackson.databind.JsonNode;
@@ -59,6 +60,22 @@
         return res;
     }
 
+    @Override
+    public ExtractedResult executeSqlppUpdateOrDdl(String statement, TestCaseContext.OutputFormat outputFormat,
+            TestCase.CompilationUnit cUnit) throws Exception {
+        Matcher dvMatcher = usePattern.matcher(statement);
+        String dv = "";
+        if (dvMatcher.find()) {
+            dv = dvMatcher.group(2) + ".";
+        }
+        Matcher dsMatcher = loadPattern.matcher(statement);
+        Matcher upsertMatcher = upsertPattern.matcher(statement);
+        ExtractedResult res = super.executeUpdateOrDdl(statement, outputFormat, getQueryServiceUri(SQLPP), cUnit);
+        analyzeFromRegex(dsMatcher, dv, 3);
+        analyzeFromRegex(upsertMatcher, dv, 2);
+        return res;
+    }
+
     private void analyzeFromRegex(Matcher m, String dv, int pos) throws Exception {
         while (m.find()) {
             String ds = m.group(pos);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/CancellationTestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/CancellationTestExecutor.java
index 766a9d4..b4b7352 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/CancellationTestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/CancellationTestExecutor.java
@@ -20,6 +20,7 @@
 package org.apache.asterix.test.common;
 
 import static org.apache.asterix.api.http.server.QueryServiceRequestParameters.Parameter.CLIENT_ID;
+import static org.apache.asterix.test.common.ComparisonException.Type.DIFFERENT_RESULT;
 
 import java.io.InputStream;
 import java.net.URI;
@@ -110,6 +111,10 @@
             SqlppExecutionWithCancellationTest.numCancelledQueries++;
             queryCount.increment();
             return false;
+        } else if (e instanceof ComparisonException
+                && ((ComparisonException) e).getExceptionType() == DIFFERENT_RESULT) {
+            // for this test, ignore ComparisonException for completed requests since the goal is not to compare results
+            return false;
         } else {
             System.err.println(
                     "Expected to find one of the following in error text:\n+++++\n" + expectedErrors + "\n+++++");
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ComparisonException.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ComparisonException.java
index cb437f0..ec3cabf 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ComparisonException.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ComparisonException.java
@@ -19,13 +19,40 @@
 package org.apache.asterix.test.common;
 
 public class ComparisonException extends Exception {
-    private static final long serialVersionUID = 1L;
 
-    public ComparisonException(String message) {
-        super(message);
+    enum Type {
+        DIFFERENT_RESULT,
+        MALFORMED_RESULT,
+        NO_RESULT
     }
 
-    public ComparisonException(String message, Throwable cause) {
+    private static final long serialVersionUID = 1L;
+
+    private final Type exceptionType;
+
+    private ComparisonException(String message, Type type) {
+        super(message);
+        exceptionType = type;
+    }
+
+    private ComparisonException(String message, Throwable cause, Type type) {
         super(message, cause);
+        exceptionType = type;
+    }
+
+    public static ComparisonException noResult(String message) {
+        return new ComparisonException(message, Type.NO_RESULT);
+    }
+
+    public static ComparisonException differentResult(String message) {
+        return new ComparisonException(message, Type.DIFFERENT_RESULT);
+    }
+
+    public static ComparisonException malformedResult(String message, Throwable cause) {
+        return new ComparisonException(message, cause, Type.MALFORMED_RESULT);
+    }
+
+    public Type getExceptionType() {
+        return exceptionType;
     }
 }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestConstants.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestConstants.java
index 1a5cb66..bef2f97 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestConstants.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestConstants.java
@@ -34,6 +34,10 @@
     public static final String S3_TEMPLATE_DEFAULT = "(\"accessKeyId\"=\"" + S3_ACCESS_KEY_ID_DEFAULT + "\"),\n"
             + "(\"secretAccessKey\"=\"" + S3_SECRET_ACCESS_KEY_DEFAULT + "\"),\n" + "(\"region\"=\"" + S3_REGION_DEFAULT
             + "\"),\n" + "(\"serviceEndpoint\"=\"" + S3_SERVICE_ENDPOINT_DEFAULT + "\")";
+    public static final String S3_TEMPLATE_DEFAULT_NO_PARENTHESES_WITH_COLONS =
+            "\"accessKeyId\":\"" + S3_ACCESS_KEY_ID_DEFAULT + "\",\n" + "\"secretAccessKey\":\""
+                    + S3_SECRET_ACCESS_KEY_DEFAULT + "\",\n" + "\"region\":\"" + S3_REGION_DEFAULT + "\",\n"
+                    + "\"serviceEndpoint\":\"" + S3_SERVICE_ENDPOINT_DEFAULT + "\"";
 
     // Azure blob storage constants and placeholders
     public static class Azure {
@@ -82,4 +86,17 @@
                 + BLOB_ENDPOINT_PLACEHOLDER + "\")";
         public static final String TEMPLATE_DEFAULT = TEMPLATE;
     }
+
+    public static class HDFS {
+        public static final String HDFS_AUTHENTICATION_DEFAULT = "kerberos";
+        public static final String KERBEROS_PRINCIPAL_DEFAULT = "hdfsuser@EXAMPLE.COM";
+        public static final String KERBEROS_PASSWORD_DEFAULT = "hdfspassword";
+        public static final String KERBEROS_REALM_DEFAULT = "EXAMPLE.COM";
+        public static final String KERBEROS_KDC_DEFAULT = "localhost:8800";
+        public static final String HDFS_ENDPOINT_DEFAULT = "hdfs://localhost:31888";
+
+        public static final String HDFS_TEMPLATE_DEFAULT = "(\"hdfs\"=\"" + HDFS_ENDPOINT_DEFAULT + "\")";
+        public static final String HDFS_TEMPLATE_DEFAULT_NO_PARENTHESES_WITH_COLONS =
+                "\"hdfs\":\"" + HDFS_ENDPOINT_DEFAULT + "\"";
+    }
 }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
index 37527bd..3d39ef4 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
@@ -424,18 +424,18 @@
 
     public static ComparisonException createLineChangedException(File scriptFile, String lineExpected,
             String lineActual, int num) {
-        return new ComparisonException("Result for " + canonicalize(scriptFile) + " changed at line " + num
+        return ComparisonException.differentResult("Result for " + canonicalize(scriptFile) + " changed at line " + num
                 + ":\nexpected < " + truncateIfLong(lineExpected) + "\nactual   > " + truncateIfLong(lineActual));
     }
 
     public static ComparisonException createLineNotFoundException(File scriptFile, String lineExpected, int num) {
-        return new ComparisonException("Result for " + canonicalize(scriptFile) + " expected line at " + num
+        return ComparisonException.differentResult("Result for " + canonicalize(scriptFile) + " expected line at " + num
                 + " not found: " + truncateIfLong(lineExpected));
     }
 
     private ComparisonException createExpectedLinesNotReturnedException(File scriptFile, List<String> expectedLines) {
-        return new ComparisonException("Result for " + canonicalize(scriptFile) + " expected lines not returned:\n"
-                + String.join("\n", expectedLines));
+        return ComparisonException.differentResult("Result for " + canonicalize(scriptFile)
+                + " expected lines not returned:\n" + String.join("\n", expectedLines));
     }
 
     private static String truncateIfLong(String string) {
@@ -621,20 +621,20 @@
         try {
             expectedJson = SINGLE_JSON_NODE_READER.readTree(readerExpected);
         } catch (JsonProcessingException e) {
-            throw new ComparisonException("Invalid expected JSON for: " + scriptFile, e);
+            throw ComparisonException.malformedResult("Invalid expected JSON for: " + scriptFile, e);
         }
         try {
             actualJson = SINGLE_JSON_NODE_READER.readTree(readerActual);
         } catch (JsonProcessingException e) {
-            throw new ComparisonException("Invalid actual JSON for: " + scriptFile, e);
+            throw ComparisonException.malformedResult("Invalid actual JSON for: " + scriptFile, e);
         }
         if (expectedJson == null) {
-            throw new ComparisonException("No expected result for: " + scriptFile);
+            throw ComparisonException.noResult("No expected result for: " + scriptFile);
         } else if (actualJson == null) {
-            throw new ComparisonException("No actual result for: " + scriptFile);
+            throw ComparisonException.noResult("No actual result for: " + scriptFile);
         }
         if (!TestHelper.equalJson(expectedJson, actualJson, compareUnorderedArray, ignoreExtraFields, false, null)) {
-            throw new ComparisonException("Result for " + scriptFile + " didn't match the expected JSON"
+            throw ComparisonException.differentResult("Result for " + scriptFile + " didn't match the expected JSON"
                     + "\nexpected result:\n" + expectedJson + "\nactual result:\n" + actualJson);
         }
     }
@@ -650,35 +650,35 @@
         int i = 0;
         for (String expectedLine : expectedLines) {
             if (actualLines.size() <= i) {
-                throw new ComparisonException("Result for " + canonicalize(scriptFile) + " expected json line at " + i
-                        + " not found: " + truncateIfLong(expectedLine));
+                throw ComparisonException.differentResult("Result for " + canonicalize(scriptFile)
+                        + " expected json line at " + i + " not found: " + truncateIfLong(expectedLine));
             }
             String actualLine = actualLines.get(i);
             i += 1;
             try {
                 expectedJson = SINGLE_JSON_NODE_READER.readTree(expectedLine);
             } catch (JsonProcessingException e) {
-                throw new ComparisonException("Invalid expected JSON for: " + scriptFile, e);
+                throw ComparisonException.malformedResult("Invalid expected JSON for: " + scriptFile, e);
             }
             try {
                 actualJson = SINGLE_JSON_NODE_READER.readTree(actualLine);
             } catch (JsonProcessingException e) {
-                throw new ComparisonException("Invalid actual JSON for: " + scriptFile, e);
+                throw ComparisonException.malformedResult("Invalid actual JSON for: " + scriptFile, e);
             }
             if (expectedJson == null) {
-                throw new ComparisonException("No expected result for: " + scriptFile);
+                throw ComparisonException.noResult("No expected result for: " + scriptFile);
             } else if (actualJson == null) {
-                throw new ComparisonException("No actual result for: " + scriptFile);
+                throw ComparisonException.noResult("No actual result for: " + scriptFile);
             }
             if (!TestHelper.equalJson(expectedJson, actualJson, compareUnorderedArray, ignoreExtraFields, false,
                     null)) {
-                throw new ComparisonException("Result for " + scriptFile + " didn't match the expected JSON"
+                throw ComparisonException.differentResult("Result for " + scriptFile + " didn't match the expected JSON"
                         + "\nexpected result:\n" + expectedJson + "\nactual result:\n" + actualJson);
             }
         }
         if (actualLines.size() > i) {
-            throw new ComparisonException("Result for " + canonicalize(scriptFile) + " extra json line at " + i
-                    + " found: " + truncateIfLong(actualLines.get(i)));
+            throw ComparisonException.differentResult("Result for " + canonicalize(scriptFile) + " extra json line at "
+                    + i + " found: " + truncateIfLong(actualLines.get(i)));
         }
     }
 
@@ -1274,7 +1274,11 @@
                 if (isDmlRecoveryTest && statement.contains("nc1://")) {
                     statement = statement.replaceAll("nc1://", "127.0.0.1://../../../../../../asterix-app/");
                 }
-                executeSqlppUpdateOrDdl(statement, OutputFormat.forCompilationUnit(cUnit));
+                if (cUnit.getPlaceholder().isEmpty()) {
+                    executeSqlppUpdateOrDdl(statement, OutputFormat.forCompilationUnit(cUnit));
+                } else {
+                    executeSqlppUpdateOrDdl(statement, OutputFormat.forCompilationUnit(cUnit), cUnit);
+                }
                 break;
             case "pollget":
             case "pollquery":
@@ -2424,6 +2428,8 @@
                     str = applyAzureSubstitution(str, placeholders);
                 } else if (placeholder.getValue().equalsIgnoreCase("GCS")) {
                     str = applyGCSSubstitution(str, placeholders);
+                } else if (placeholder.getValue().equalsIgnoreCase("HDFS")) {
+                    str = applyHDFSSubstitution(str, placeholders);
                 }
             } else {
                 // Any other place holders, just replace with the value
@@ -2451,7 +2457,7 @@
     }
 
     protected boolean noTemplateRequired(String str) {
-        return !str.contains("%template%");
+        return !str.contains("%template%") && !str.contains("%template_colons%");
     }
 
     protected String applyS3Substitution(String str, List<Placeholder> placeholders) {
@@ -2502,7 +2508,11 @@
     }
 
     protected String setS3TemplateDefault(String str) {
-        return str.replace("%template%", TestConstants.S3_TEMPLATE_DEFAULT);
+        if (str.contains("%template%")) {
+            return str.replace("%template%", TestConstants.S3_TEMPLATE_DEFAULT);
+        } else {
+            return str.replace("%template_colons%", TestConstants.S3_TEMPLATE_DEFAULT_NO_PARENTHESES_WITH_COLONS);
+        }
     }
 
     protected String applyAzureSubstitution(String str, List<Placeholder> placeholders) {
@@ -2534,6 +2544,11 @@
         return str;
     }
 
+    protected String applyHDFSSubstitution(String str, List<Placeholder> placeholders) {
+        str = setHDFSTemplateDefault(str);
+        return str;
+    }
+
     protected String setAzureTemplate(String str) {
         return str.replace("%template%", TEMPLATE);
     }
@@ -2546,6 +2561,15 @@
         return str;
     }
 
+    protected String setHDFSTemplateDefault(String str) {
+        if (str.contains("%template%")) {
+            return str.replace("%template%", TestConstants.HDFS.HDFS_TEMPLATE_DEFAULT);
+        } else {
+            return str.replace("%template_colons%",
+                    TestConstants.HDFS.HDFS_TEMPLATE_DEFAULT_NO_PARENTHESES_WITH_COLONS);
+        }
+    }
+
     protected void fail(boolean runDiagnostics, TestCaseContext testCaseCtx, CompilationUnit cUnit,
             List<TestFileContext> testFileCtxs, ProcessBuilder pb, File testFile, Exception e) throws Exception {
         if (runDiagnostics) {
@@ -3011,7 +3035,7 @@
                         LOGGER.error("was expecting the following warnings: ");
                     }
                     for (int i = expectedWarnings.nextSetBit(0); i >= 0; i = expectedWarnings.nextSetBit(i + 1)) {
-                        LOGGER.error(expectedWarn.get(i));
+                        LOGGER.error(expectedWarn.get(i).getValue());
                     }
                     throw new Exception(msg);
                 }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java
index 70226c8..4024ced 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java
@@ -51,6 +51,7 @@
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails.PartitioningStrategy;
+import org.apache.asterix.metadata.utils.Creator;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
@@ -174,7 +175,8 @@
                 null, DatasetType.INTERNAL, DATASET_ID, 0);
         String database = MetadataUtil.databaseFor(dvName);
         secondaryIndex = new Index(database, dvName, DATASET_NAME, INDEX_NAME, INDEX_TYPE, INDEX_FIELD_NAMES,
-                INDEX_FIELD_INDICATORS, INDEX_FIELD_TYPES, false, false, false, 0, OptionalBoolean.of(false));
+                INDEX_FIELD_INDICATORS, INDEX_FIELD_TYPES, false, false, false, 0, OptionalBoolean.of(false),
+                Creator.DEFAULT_CREATOR);
         taskCtx = null;
         primaryIndexDataflowHelper = null;
         secondaryIndexDataflowHelper = null;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ConcurrentInsertTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ConcurrentInsertTest.java
index b458874..b85cae2 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ConcurrentInsertTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ConcurrentInsertTest.java
@@ -130,7 +130,7 @@
                 StorageTestUtils.DATASET.getDataverseName(), StorageTestUtils.DATASET.getDatasetName(), "TestIndex",
                 IndexType.BTREE, Arrays.asList(Arrays.asList(StorageTestUtils.RECORD_TYPE.getFieldNames()[1])),
                 Arrays.asList(Index.RECORD_INDICATOR), Arrays.asList(BuiltinType.AINT64), false, false, false, 0,
-                OptionalBoolean.of(false));
+                OptionalBoolean.of(false), StorageTestUtils.DATASET.getCreator());
 
         SecondaryIndexInfo secondaryIndexInfo =
                 nc.createSecondaryIndex(primaryIndexInfo, secondaryIndexEntity, StorageTestUtils.STORAGE_MANAGER, 0);
@@ -142,10 +142,10 @@
         secondaryIndex = (TestLsmBtree) secondaryDataflowHelper.getIndexInstance();
         secondaryDataflowHelper.close();
 
-        Index primaryKeyIndexEntity =
-                new Index(StorageTestUtils.DATASET.getDatabaseName(), StorageTestUtils.DATASET.getDataverseName(),
-                        StorageTestUtils.DATASET.getDatasetName(), "PrimaryKeyIndex", IndexType.BTREE, Arrays.asList(),
-                        Arrays.asList(), Arrays.asList(), false, false, false, 0, OptionalBoolean.empty());
+        Index primaryKeyIndexEntity = new Index(StorageTestUtils.DATASET.getDatabaseName(),
+                StorageTestUtils.DATASET.getDataverseName(), StorageTestUtils.DATASET.getDatasetName(),
+                "PrimaryKeyIndex", IndexType.BTREE, Arrays.asList(), Arrays.asList(), Arrays.asList(), false, false,
+                false, 0, OptionalBoolean.empty(), StorageTestUtils.DATASET.getCreator());
 
         SecondaryIndexInfo primaryKeyIndexInfo =
                 nc.createSecondaryIndex(primaryIndexInfo, primaryKeyIndexEntity, StorageTestUtils.STORAGE_MANAGER, 0);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java
index b574c1b..7c1a534 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java
@@ -189,7 +189,7 @@
         secondaryIndexEntity = new Index(dataset.getDatabaseName(), dataset.getDataverseName(),
                 dataset.getDatasetName(), SECONDARY_INDEX_NAME, SECONDARY_INDEX_TYPE, SECONDARY_INDEX_FIELD_NAMES,
                 SECONDARY_INDEX_FIELD_INDICATORS, SECONDARY_INDEX_FIELD_TYPES, false, false, false, 0,
-                OptionalBoolean.of(false));
+                OptionalBoolean.of(false), dataset.getCreator());
 
         primaryIndexInfos = new PrimaryIndexInfo[NUM_PARTITIONS];
         secondaryIndexInfo = new SecondaryIndexInfo[NUM_PARTITIONS];
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java
index d921127..cc50f2a 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java
@@ -161,9 +161,9 @@
                 NoMergePolicyFactory.NAME, null, new InternalDatasetDetails(null, PartitioningStrategy.HASH,
                         partitioningKeys, null, null, null, false, null, null),
                 null, DatasetType.INTERNAL, DATASET_ID, 0);
-        secondaryIndex =
-                new Index(dataset.getDatabaseName(), dvName, DATASET_NAME, INDEX_NAME, INDEX_TYPE, INDEX_FIELD_NAMES,
-                        INDEX_FIELD_INDICATORS, INDEX_FIELD_TYPES, false, false, false, 0, OptionalBoolean.of(false));
+        secondaryIndex = new Index(dataset.getDatabaseName(), dvName, DATASET_NAME, INDEX_NAME, INDEX_TYPE,
+                INDEX_FIELD_NAMES, INDEX_FIELD_INDICATORS, INDEX_FIELD_TYPES, false, false, false, 0,
+                OptionalBoolean.of(false), dataset.getCreator());
         taskCtxs = new IHyracksTaskContext[NUM_PARTITIONS];
         primaryIndexDataflowHelpers = new IIndexDataflowHelper[NUM_PARTITIONS];
         secondaryIndexDataflowHelpers = new IIndexDataflowHelper[NUM_PARTITIONS];
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java
index 210440d..b268b19 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java
@@ -28,6 +28,7 @@
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.Creator;
 import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.transaction.management.resource.DatasetLocalResourceFactory;
@@ -54,7 +55,7 @@
         super(MetadataUtil.databaseFor(dataverseName), dataverseName, datasetName,
                 MetadataUtil.databaseFor(recordTypeDataverseName), recordTypeDataverseName, recordTypeName,
                 nodeGroupName, compactionPolicy, compactionPolicyProperties, datasetDetails, hints, datasetType,
-                datasetId, pendingOp);
+                datasetId, pendingOp, Creator.DEFAULT_CREATOR);
     }
 
     @Override
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/ExternalDatasetTestUtils.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/ExternalDatasetTestUtils.java
index 9e63c44..c94fd60 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/ExternalDatasetTestUtils.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/ExternalDatasetTestUtils.java
@@ -441,6 +441,11 @@
         loadDeltaDirectory(generatedDataBasePath, "/multiple_file_delta_table/_delta_log", JSON_FILTER, "delta-data/");
         loadDeltaDirectory(generatedDataBasePath, "/delta_all_type/_delta_log", JSON_FILTER, "delta-data/");
         loadDeltaDirectory(generatedDataBasePath, "/delta_all_type", PARQUET_FILTER, "delta-data/");
+        loadDeltaDirectory(generatedDataBasePath, "/delta_file_size_nine/_delta_log", JSON_FILTER, "delta-data/");
+        loadDeltaDirectory(generatedDataBasePath, "/delta_file_size_nine", PARQUET_FILTER, "delta-data/");
+        loadDeltaDirectory(generatedDataBasePath, "/delta_file_size_one/_delta_log", JSON_FILTER, "delta-data/");
+        loadDeltaDirectory(generatedDataBasePath, "/delta_file_size_one", PARQUET_FILTER, "delta-data/");
+
     }
 
     private static void loadDeltaDirectory(String dataBasePath, String rootPath, FilenameFilter filter,
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/deltalake/DeltaTableGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/deltalake/DeltaTableGenerator.java
index 1236636..67d460c 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/deltalake/DeltaTableGenerator.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/deltalake/DeltaTableGenerator.java
@@ -55,6 +55,10 @@
             "target" + File.separatorChar + "generated_delta_files" + File.separatorChar + "modified_delta_table";
     public static final String DELTA_MULTI_FILE_TABLE =
             "target" + File.separatorChar + "generated_delta_files" + File.separatorChar + "multiple_file_delta_table";
+    public static final String DELTA_FILE_SIZE_ONE =
+            "target" + File.separatorChar + "generated_delta_files" + File.separatorChar + "delta_file_size_one";
+    public static final String DELTA_FILE_SIZE_NINE =
+            "target" + File.separatorChar + "generated_delta_files" + File.separatorChar + "delta_file_size_nine";
 
     public static void prepareDeltaTableContainer(Configuration conf) {
         File basePath = new File(".");
@@ -62,6 +66,8 @@
         prepareMultipleFilesTable(conf);
         prepareModifiedTable(conf);
         prepareEmptyTable(conf);
+        prepareFileSizeOne(conf);
+        prepareFileSizeNine(conf);
     }
 
     public static void cleanBinaryDirectory(File localDataRoot, String binaryFilesPath) {
@@ -221,4 +227,129 @@
             throw new RuntimeException(e);
         }
     }
+
+    public static void prepareFileSizeOne(Configuration conf) {
+        Schema schema = SchemaBuilder.record("MyRecord").fields().requiredInt("id").requiredString("name").endRecord();
+        try {
+            Path path = new Path(DELTA_FILE_SIZE_ONE, "firstFile.parquet");
+            ParquetWriter<GenericData.Record> writer =
+                    AvroParquetWriter.<GenericData.Record> builder(path).withConf(conf).withSchema(schema).build();
+
+            List<GenericData.Record> fileFirstSnapshotRecords = List.of(new GenericData.Record(schema));
+
+            fileFirstSnapshotRecords.get(0).put("id", 0);
+            fileFirstSnapshotRecords.get(0).put("name", "Cooper");
+
+            for (GenericData.Record record : fileFirstSnapshotRecords) {
+                writer.write(record);
+            }
+
+            long size = writer.getDataSize();
+            writer.close();
+
+            List<Action> actions = List.of(new AddFile("firstFile.parquet", new HashMap<>(), size,
+                    System.currentTimeMillis(), true, null, null));
+            DeltaLog log = DeltaLog.forTable(conf, DELTA_FILE_SIZE_ONE);
+            OptimisticTransaction txn = log.startTransaction();
+            Metadata metaData = txn.metadata().copyBuilder().partitionColumns(new ArrayList<>())
+                    .schema(new StructType().add(new StructField("id", new IntegerType(), true))
+                            .add(new StructField("name", new StringType(), true)))
+                    .build();
+            txn.updateMetadata(metaData);
+            txn.commit(actions, new Operation(Operation.Name.CREATE_TABLE), "deltalake-table-create");
+
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static void prepareFileSizeNine(Configuration conf) {
+        Schema schema = SchemaBuilder.record("MyRecord").fields().requiredInt("id").requiredString("name").endRecord();
+        try {
+            Path path = new Path(DELTA_FILE_SIZE_NINE, "firstFile.parquet");
+            ParquetWriter<GenericData.Record> writer =
+                    AvroParquetWriter.<GenericData.Record> builder(path).withConf(conf).withSchema(schema).build();
+
+            List<GenericData.Record> fileFirstSnapshotRecords = List.of(new GenericData.Record(schema));
+            List<GenericData.Record> fileSecondSnapshotRecords = List.of(new GenericData.Record(schema));
+            List<GenericData.Record> fileThirdSnapshotRecords = List.of(new GenericData.Record(schema));
+            List<GenericData.Record> fileFourthSnapshotRecords = List.of(new GenericData.Record(schema));
+            List<GenericData.Record> fileFifthSnapshotRecords = List.of(new GenericData.Record(schema));
+            List<GenericData.Record> fileSixthSnapshotRecords = List.of(new GenericData.Record(schema));
+            List<GenericData.Record> fileSeventhSnapshotRecords = List.of(new GenericData.Record(schema));
+            List<GenericData.Record> fileEightSnapshotRecords = List.of(new GenericData.Record(schema));
+            List<GenericData.Record> fileNineSnapshotRecords = List.of(new GenericData.Record(schema));
+
+            List<List<GenericData.Record>> allSnapshotRecords =
+                    List.of(fileFirstSnapshotRecords, fileSecondSnapshotRecords, fileThirdSnapshotRecords,
+                            fileFourthSnapshotRecords, fileFifthSnapshotRecords, fileSixthSnapshotRecords,
+                            fileSeventhSnapshotRecords, fileEightSnapshotRecords, fileNineSnapshotRecords);
+
+            fileFirstSnapshotRecords.get(0).put("id", 0);
+            fileFirstSnapshotRecords.get(0).put("name", "Cooper");
+
+            fileSecondSnapshotRecords.get(0).put("id", 1);
+            fileSecondSnapshotRecords.get(0).put("name", "Adam");
+
+            fileThirdSnapshotRecords.get(0).put("id", 2);
+            fileThirdSnapshotRecords.get(0).put("name", "Third");
+
+            fileFourthSnapshotRecords.get(0).put("id", 3);
+            fileFourthSnapshotRecords.get(0).put("name", "Fourth");
+
+            fileFifthSnapshotRecords.get(0).put("id", 4);
+            fileFifthSnapshotRecords.get(0).put("name", "Five");
+
+            fileSixthSnapshotRecords.get(0).put("id", 5);
+            fileSixthSnapshotRecords.get(0).put("name", "Six");
+
+            fileSeventhSnapshotRecords.get(0).put("id", 6);
+            fileSeventhSnapshotRecords.get(0).put("name", "Seven");
+
+            fileEightSnapshotRecords.get(0).put("id", 7);
+            fileEightSnapshotRecords.get(0).put("name", "Eight");
+
+            fileNineSnapshotRecords.get(0).put("id", 8);
+            fileNineSnapshotRecords.get(0).put("name", "Nine");
+
+            for (GenericData.Record record : fileFirstSnapshotRecords) {
+                writer.write(record);
+            }
+
+            long size = writer.getDataSize();
+            writer.close();
+
+            List<Action> actions = List.of(new AddFile("firstFile.parquet", new HashMap<>(), size,
+                    System.currentTimeMillis(), true, null, null));
+            DeltaLog log = DeltaLog.forTable(conf, DELTA_FILE_SIZE_NINE);
+            OptimisticTransaction txn = log.startTransaction();
+            Metadata metaData = txn.metadata().copyBuilder().partitionColumns(new ArrayList<>())
+                    .schema(new StructType().add(new StructField("id", new IntegerType(), true))
+                            .add(new StructField("name", new StringType(), true)))
+                    .build();
+            txn.updateMetadata(metaData);
+            txn.commit(actions, new Operation(Operation.Name.CREATE_TABLE), "deltalake-table-create");
+
+            for (int i = 2; i <= 9; i++) {
+                Path path2 = new Path(DELTA_FILE_SIZE_NINE, "File" + i + ".parquet");
+                ParquetWriter<GenericData.Record> writer2 =
+                        AvroParquetWriter.<GenericData.Record> builder(path2).withConf(conf).withSchema(schema).build();
+
+                for (GenericData.Record record : allSnapshotRecords.get(i - 1)) {
+                    writer2.write(record);
+                }
+
+                long size2 = writer2.getDataSize();
+                writer2.close();
+
+                List<Action> actions2 = List.of(new AddFile("File" + i + ".parquet", new HashMap<>(), size2,
+                        System.currentTimeMillis(), true, null, null));
+
+                OptimisticTransaction txn2 = log.startTransaction();
+                txn2.commit(actions2, new Operation(Operation.Name.WRITE), "deltalake-table-create");
+            }
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
 }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java
index 0b66676..0d3102a 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java
@@ -325,7 +325,7 @@
         Dataset dataset = new Dataset(source.getDatabaseName(), source.getDataverseName(), "ds_" + datasetPostfix,
                 source.getDatabaseName(), source.getDataverseName(), source.getDatasetType().name(),
                 source.getNodeGroupName(), NoMergePolicyFactory.NAME, null, source.getDatasetDetails(),
-                source.getHints(), DatasetConfig.DatasetType.INTERNAL, datasetPostfix, 0);
+                source.getHints(), DatasetConfig.DatasetType.INTERNAL, datasetPostfix, 0, source.getCreator());
         MetadataProvider metadataProvider = MetadataProvider.createWithDefaultNamespace(appCtx);
         final MetadataTransactionContext writeTxn = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(writeTxn);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/NonCloudSqlppExecutionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/NonCloudSqlppExecutionTest.java
new file mode 100644
index 0000000..c403178
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/NonCloudSqlppExecutionTest.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.runtime;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.api.common.LocalCloudUtil;
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.hyracks.control.nc.NodeControllerService;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Runs the SQL++ runtime tests with the storage parallelism.
+ * This version of the tests runs the tests that aren't meant to be run in the cloud because they can have different
+ * output.
+ */
+@RunWith(Parameterized.class)
+public class NonCloudSqlppExecutionTest {
+    protected static final String TEST_CONFIG_FILE_NAME = "src/main/resources/cc.conf";
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        final TestExecutor testExecutor = new TestExecutor();
+        LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, testExecutor);
+        setNcEndpoints(testExecutor);
+        LocalCloudUtil.startS3CloudEnvironment(true, true);
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        LangExecutionUtil.tearDown();
+    }
+
+    @Parameters(name = "NonCloudSqlppExecutionTest {index}: {0}")
+    public static Collection<Object[]> tests() throws Exception {
+        return LangExecutionUtil.tests("only_sqlpp.xml", "non_cloud_sqlpp_queries.xml");
+    }
+
+    protected TestCaseContext tcCtx;
+
+    public NonCloudSqlppExecutionTest(TestCaseContext tcCtx) {
+        this.tcCtx = tcCtx;
+    }
+
+    @Test
+    public void test() throws Exception {
+        LangExecutionUtil.test(tcCtx);
+    }
+
+    private static void setNcEndpoints(TestExecutor testExecutor) {
+        final NodeControllerService[] ncs = ExecutionTestUtil.integrationUtil.ncs;
+        final Map<String, InetSocketAddress> ncEndPoints = new HashMap<>();
+        final String ip = InetAddress.getLoopbackAddress().getHostAddress();
+        for (NodeControllerService nc : ncs) {
+            final String nodeId = nc.getId();
+            final INcApplicationContext appCtx = (INcApplicationContext) nc.getApplicationContext();
+            int apiPort = appCtx.getExternalProperties().getNcApiPort();
+            ncEndPoints.put(nodeId, InetSocketAddress.createUnresolved(ip, apiPort));
+        }
+        testExecutor.setNcEndPoints(ncEndPoints);
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppHdfsExecutionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppHdfsExecutionTest.java
index fe3006e..412fbc1 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppHdfsExecutionTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppHdfsExecutionTest.java
@@ -18,17 +18,36 @@
  */
 package org.apache.asterix.test.runtime;
 
+import static org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils.createAvroFiles;
+import static org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils.createAvroFilesRecursively;
+import static org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils.createBinaryFiles;
+import static org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils.createBinaryFilesRecursively;
+import static org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils.createDeltaTable;
+import static org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils.setDataPaths;
+import static org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils.setUploaders;
+import static org.apache.asterix.test.external_dataset.parquet.BinaryFileConverterUtil.DEFAULT_PARQUET_SRC_PATH;
+import static org.apache.hyracks.util.file.FileUtil.joinPath;
 import static org.apache.iceberg.hadoop.HadoopOutputFile.fromPath;
 import static org.apache.iceberg.types.Types.NestedField.required;
 
+import java.io.FileInputStream;
 import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Paths;
 import java.util.Collection;
 import java.util.List;
+import java.util.zip.GZIPOutputStream;
 
 import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.test.common.TestConstants;
 import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils;
 import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.iceberg.DataFile;
 import org.apache.iceberg.DataFiles;
@@ -60,34 +79,42 @@
  */
 @RunWith(Parameterized.class)
 public class SqlppHdfsExecutionTest {
+    private static final String PATH_BASE = joinPath("data");
+    private static final String EXTERNAL_FILTER_DATA_PATH = joinPath(PATH_BASE, "json", "external-filter");
+
     protected static final String TEST_CONFIG_FILE_NAME = "src/main/resources/cc.conf";
 
-    private static DataFile writeFile(String filename, List<Record> records, String location, Schema schema,
-            Configuration conf) throws IOException {
+    static Runnable PREPARE_BUCKET;
+
+    private static final String JSON_DATA_PATH = joinPath("data", "json");
+    private static final String CSV_DATA_PATH = joinPath("data", "csv");
+    private static final String TSV_DATA_PATH = joinPath("data", "tsv");
+
+    private static final Configuration CONF = new Configuration();
+
+    private static DataFile writeFile(String filename, List<Record> records, String location, Schema schema)
+            throws IOException {
         Path path = new Path(location, filename);
         FileFormat fileFormat = FileFormat.fromFileName(filename);
         Preconditions.checkNotNull(fileFormat, "Cannot determine format for file: %s", filename);
 
         FileAppender<Record> fileAppender =
-                new GenericAppenderFactory(schema).newAppender(fromPath(path, conf), fileFormat);
+                new GenericAppenderFactory(schema).newAppender(fromPath(path, CONF), fileFormat);
         try (FileAppender<Record> appender = fileAppender) {
             appender.addAll(records);
         }
 
-        return DataFiles.builder(PartitionSpec.unpartitioned()).withInputFile(HadoopInputFile.fromPath(path, conf))
+        return DataFiles.builder(PartitionSpec.unpartitioned()).withInputFile(HadoopInputFile.fromPath(path, CONF))
                 .withMetrics(fileAppender.metrics()).build();
     }
 
     private static void setUpIcebergData() {
-        Configuration conf = new Configuration();
-        conf.set(ExternalDataConstants.KEY_HADOOP_FILESYSTEM_URI, "hdfs://127.0.0.1:31888/");
-
-        Tables tables = new HadoopTables(conf);
+        Tables tables = new HadoopTables(CONF);
 
         Schema schema =
                 new Schema(required(1, "id", Types.IntegerType.get()), required(2, "data", Types.StringType.get()));
 
-        String path = "hdfs://localhost:31888/my_table/";
+        String path = TestConstants.HDFS.HDFS_ENDPOINT_DEFAULT + "/my_table/";
 
         Table table = tables.create(schema, PartitionSpec.unpartitioned(),
                 ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, FileFormat.PARQUET.name()), path);
@@ -100,8 +127,7 @@
 
         // load test data
         try {
-            DataFile file =
-                    writeFile(FileFormat.PARQUET.addExtension("file"), fileFirstSnapshotRecords, path, schema, conf);
+            DataFile file = writeFile(FileFormat.PARQUET.addExtension("file"), fileFirstSnapshotRecords, path, schema);
             table.newAppend().appendFile(file).commit();
         } catch (IOException e) {
             throw new RuntimeException(e);
@@ -110,8 +136,52 @@
 
     @BeforeClass
     public static void setUp() throws Exception {
+        CONF.set(ExternalDataConstants.KEY_HADOOP_FILESYSTEM_URI, TestConstants.HDFS.HDFS_ENDPOINT_DEFAULT);
         LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, new TestExecutor(), true);
         setUpIcebergData();
+        createBinaryFiles(DEFAULT_PARQUET_SRC_PATH);
+        createBinaryFilesRecursively(EXTERNAL_FILTER_DATA_PATH);
+        createAvroFiles(DEFAULT_PARQUET_SRC_PATH);
+        createAvroFilesRecursively(EXTERNAL_FILTER_DATA_PATH);
+        createDeltaTable();
+        setUpData();
+    }
+
+    private static void setUpData() {
+        setDataPaths(JSON_DATA_PATH, CSV_DATA_PATH, TSV_DATA_PATH);
+        setUploaders(SqlppHdfsExecutionTest::loadPlaygroundData, null, null, null, null);
+
+        PREPARE_BUCKET.run();
+    }
+
+    private static void loadPlaygroundData(String key, String content, boolean fromFile, boolean gzipped) {
+        loadData("/playground/", key, content, fromFile, gzipped);
+    }
+
+    private static void loadData(String prefix, String key, String content, boolean fromFile, boolean gzipped) {
+        try {
+            try (FileSystem fs = FileSystem.get(CONF)) {
+                Path path = new Path(prefix + key);
+                if (!fromFile) {
+                    try (FSDataOutputStream out = fs.create(path)) {
+                        out.writeBytes(content);
+                    }
+                } else {
+                    if (!gzipped) {
+                        try (FSDataOutputStream out = fs.create(path); InputStream in = new FileInputStream(content)) {
+                            IOUtils.copy(in, out);
+                        }
+                    } else {
+                        try (FSDataOutputStream out = fs.create(path);
+                                GZIPOutputStream gzipOutputStream = new GZIPOutputStream(out)) {
+                            gzipOutputStream.write(Files.readAllBytes(Paths.get(content)));
+                        }
+                    }
+                }
+            }
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
     }
 
     @AfterClass
@@ -121,6 +191,7 @@
 
     @Parameters(name = "SqlppHdfsExecutionTest {index}: {0}")
     public static Collection<Object[]> tests() throws Exception {
+        PREPARE_BUCKET = ExternalDatasetTestUtils::preparePlaygroundContainer;
         return LangExecutionUtil.tests("only_sqlpp_hdfs.xml", "testsuite_sqlpp_hdfs.xml");
     }
 
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
index 7effe9d..b8f6698 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
@@ -99,8 +99,8 @@
                 try {
                     if (queryCount >= expectedResultFileCtxs.size()
                             && !cUnit.getOutputDir().getValue().equals("none")) {
-                        throw new ComparisonException("no result file for " + canonicalize(testFile) + "; queryCount: "
-                                + queryCount + ", filectxs.size: " + expectedResultFileCtxs.size());
+                        throw ComparisonException.noResult("no result file for " + canonicalize(testFile)
+                                + "; queryCount: " + queryCount + ", filectxs.size: " + expectedResultFileCtxs.size());
                     }
 
                     // Runs the test query.
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/IndexDropOperatorNodePushableTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/IndexDropOperatorNodePushableTest.java
index 8d9a5f5..297f565 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/IndexDropOperatorNodePushableTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/IndexDropOperatorNodePushableTest.java
@@ -41,6 +41,7 @@
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
+import org.apache.asterix.metadata.utils.Creator;
 import org.apache.asterix.metadata.utils.SplitsAndConstraintsUtil;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.BuiltinType;
@@ -114,7 +115,7 @@
                     NODE_GROUP_NAME, NoMergePolicyFactory.NAME, null,
                     new InternalDatasetDetails(null, InternalDatasetDetails.PartitioningStrategy.HASH, partitioningKeys,
                             null, null, null, false, null, null),
-                    null, DatasetConfig.DatasetType.INTERNAL, DATASET_ID, 0);
+                    null, DatasetConfig.DatasetType.INTERNAL, DATASET_ID, 0, Creator.DEFAULT_CREATOR);
             // create dataset
             TestNodeController.PrimaryIndexInfo indexInfo = nc.createPrimaryIndex(dataset, KEY_TYPES, RECORD_TYPE,
                     META_TYPE, null, storageManager, KEY_INDEXES, KEY_INDICATORS_LIST, 0);
diff --git a/asterixdb/asterix-app/src/test/resources/cc-analyze.conf b/asterixdb/asterix-app/src/test/resources/cc-analyze.conf
index 93a4d15..c65bca8 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-analyze.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-analyze.conf
@@ -54,4 +54,5 @@
 log.level = INFO
 compiler.groupmemory=64MB
 storage.buffercache.pagesize=32KB
-compiler.internal.sanitycheck=true
\ No newline at end of file
+compiler.internal.sanitycheck=true
+compiler.ordered.fields=false
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/cc-cbotest.conf b/asterixdb/asterix-app/src/test/resources/cc-cbotest.conf
index 3a1ac4f..aa7e6a2 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-cbotest.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-cbotest.conf
@@ -62,6 +62,7 @@
 compiler.textsearchmemory=160KB
 compiler.windowmemory=192KB
 compiler.sort.parallel=false
+compiler.ordered.fields=false
 compiler.internal.sanitycheck=true
 messaging.frame.size=4096
 messaging.frame.count=512
diff --git a/asterixdb/asterix-app/src/test/resources/cc-cloud-storage-azblob.conf b/asterixdb/asterix-app/src/test/resources/cc-cloud-storage-azblob.conf
new file mode 100644
index 0000000..4af4fd0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/cc-cloud-storage-azblob.conf
@@ -0,0 +1,73 @@
+; Licensed to the Apache Software Foundation (ASF) under one
+; or more contributor license agreements.  See the NOTICE file
+; distributed with this work for additional information
+; regarding copyright ownership.  The ASF licenses this file
+; to you under the Apache License, Version 2.0 (the
+; "License"); you may not use this file except in compliance
+; with the License.  You may obtain a copy of the License at
+;
+;   http://www.apache.org/licenses/LICENSE-2.0
+;
+; Unless required by applicable law or agreed to in writing,
+; software distributed under the License is distributed on an
+; "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+; KIND, either express or implied.  See the License for the
+; specific language governing permissions and limitations
+; under the License.
+
+[nc/asterix_nc1]
+txn.log.dir=target/tmp/asterix_nc1/txnlog
+core.dump.dir=target/tmp/asterix_nc1/coredump
+iodevices=target/tmp/asterix_nc1/iodevice1
+iodevices=../asterix-server/target/tmp/asterix_nc1/iodevice2
+nc.api.port=19004
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5006
+
+[nc/asterix_nc2]
+ncservice.port=9091
+txn.log.dir=target/tmp/asterix_nc2/txnlog
+core.dump.dir=target/tmp/asterix_nc2/coredump
+iodevices=target/tmp/asterix_nc2/iodevice1,../asterix-server/target/tmp/asterix_nc2/iodevice2
+nc.api.port=19005
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5007
+
+[nc]
+credential.file=src/test/resources/security/passwd
+python.cmd.autolocate=true
+python.env=FOO=BAR=BAZ,BAR=BAZ
+address=127.0.0.1
+command=asterixnc
+app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
+jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
+storage.buffercache.size=128MB
+storage.memorycomponent.globalbudget=512MB
+
+[cc]
+address = 127.0.0.1
+app.class=org.apache.asterix.hyracks.bootstrap.CCApplication
+heartbeat.period=2000
+heartbeat.max.misses=25
+credential.file=src/test/resources/security/passwd
+
+[common]
+log.dir = logs/
+log.level = INFO
+compiler.framesize=32KB
+compiler.sortmemory=320KB
+compiler.groupmemory=160KB
+compiler.joinmemory=256KB
+compiler.textsearchmemory=160KB
+compiler.windowmemory=192KB
+compiler.ordered.fields=false
+compiler.internal.sanitycheck=true
+messaging.frame.size=4096
+messaging.frame.count=512
+cloud.deployment=true
+storage.buffercache.pagesize=32KB
+storage.partitioning=static
+cloud.storage.scheme=azblob
+cloud.storage.bucket=cloud-storage-container
+cloud.storage.region=us-east-2
+cloud.storage.endpoint=http://127.0.0.1:15055
+cloud.storage.anonymous.auth=true
+cloud.storage.cache.policy=lazy
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/cc-cloud-storage-gcs.conf b/asterixdb/asterix-app/src/test/resources/cc-cloud-storage-gcs.conf
index 55caad6..3251439 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-cloud-storage-gcs.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-cloud-storage-gcs.conf
@@ -58,6 +58,7 @@
 compiler.joinmemory=256KB
 compiler.textsearchmemory=160KB
 compiler.windowmemory=192KB
+compiler.ordered.fields=false
 compiler.internal.sanitycheck=true
 messaging.frame.size=4096
 messaging.frame.count=512
@@ -70,3 +71,6 @@
 cloud.storage.endpoint=http://127.0.0.1:24443
 cloud.storage.anonymous.auth=true
 cloud.storage.cache.policy=selective
+cloud.max.write.requests.per.second=1000
+cloud.max.read.requests.per.second=5000
+cloud.write.buffer.size=5
diff --git a/asterixdb/asterix-app/src/test/resources/cc-cloud-storage-main.conf b/asterixdb/asterix-app/src/test/resources/cc-cloud-storage-main.conf
new file mode 100644
index 0000000..a9f2aac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/cc-cloud-storage-main.conf
@@ -0,0 +1,74 @@
+; Licensed to the Apache Software Foundation (ASF) under one
+; or more contributor license agreements.  See the NOTICE file
+; distributed with this work for additional information
+; regarding copyright ownership.  The ASF licenses this file
+; to you under the Apache License, Version 2.0 (the
+; "License"); you may not use this file except in compliance
+; with the License.  You may obtain a copy of the License at
+;
+;   http://www.apache.org/licenses/LICENSE-2.0
+;
+; Unless required by applicable law or agreed to in writing,
+; software distributed under the License is distributed on an
+; "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+; KIND, either express or implied.  See the License for the
+; specific language governing permissions and limitations
+; under the License.
+
+[nc/asterix_nc1]
+txn.log.dir=target/tmp/asterix_nc1/txnlog
+core.dump.dir=target/tmp/asterix_nc1/coredump
+iodevices=target/tmp/asterix_nc1/iodevice1
+iodevices=../asterix-server/target/tmp/asterix_nc1/iodevice2
+nc.api.port=19004
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5006
+
+[nc/asterix_nc2]
+ncservice.port=9091
+txn.log.dir=target/tmp/asterix_nc2/txnlog
+core.dump.dir=target/tmp/asterix_nc2/coredump
+iodevices=target/tmp/asterix_nc2/iodevice1,../asterix-server/target/tmp/asterix_nc2/iodevice2
+nc.api.port=19005
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5007
+
+[nc]
+credential.file=src/test/resources/security/passwd
+python.cmd.autolocate=true
+python.env=FOO=BAR=BAZ,BAR=BAZ
+address=127.0.0.1
+command=asterixnc
+app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
+jvm.args=-Xmx4096m --add-opens=jdk.management/com.sun.management.internal=ALL-UNNAMED --add-opens=java.management/sun.management=ALL-UNNAMED --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
+storage.buffercache.size=128MB
+storage.memorycomponent.globalbudget=512MB
+
+[cc]
+address = 127.0.0.1
+app.class=org.apache.asterix.hyracks.bootstrap.CCApplication
+heartbeat.period=2000
+heartbeat.max.misses=25
+credential.file=src/test/resources/security/passwd
+
+[common]
+log.dir = logs/
+log.level = INFO
+compiler.framesize=32KB
+compiler.sortmemory=320KB
+compiler.groupmemory=160KB
+compiler.joinmemory=256KB
+compiler.textsearchmemory=160KB
+compiler.windowmemory=192KB
+compiler.internal.sanitycheck=true
+messaging.frame.size=4096
+messaging.frame.count=512
+cloud.deployment=true
+storage.buffercache.pagesize=32KB
+storage.partitioning=static
+cloud.storage.scheme=s3
+cloud.storage.bucket=cloud-storage-container
+cloud.storage.region=us-west-2
+cloud.storage.endpoint=http://127.0.0.1:8001
+cloud.storage.anonymous.auth=true
+cloud.storage.cache.policy=selective
+cloud.max.write.requests.per.second=2000
+cloud.max.read.requests.per.second=4000
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/cc-cloud-storage.conf b/asterixdb/asterix-app/src/test/resources/cc-cloud-storage.conf
index 5672cde..64fc40f 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-cloud-storage.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-cloud-storage.conf
@@ -58,6 +58,7 @@
 compiler.joinmemory=256KB
 compiler.textsearchmemory=160KB
 compiler.windowmemory=192KB
+compiler.ordered.fields=false
 compiler.internal.sanitycheck=true
 messaging.frame.size=4096
 messaging.frame.count=512
diff --git a/asterixdb/asterix-app/src/test/resources/cc-columnar.conf b/asterixdb/asterix-app/src/test/resources/cc-columnar.conf
index 11feecc..e01bbbd 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-columnar.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-columnar.conf
@@ -57,6 +57,7 @@
 compiler.joinmemory=256KB
 compiler.textsearchmemory=160KB
 compiler.windowmemory=192KB
+compiler.ordered.fields=false
 messaging.frame.size=4096
 messaging.frame.count=512
 storage.buffercache.pagesize=32KB
diff --git a/asterixdb/asterix-app/src/test/resources/cc-main.conf b/asterixdb/asterix-app/src/test/resources/cc-main.conf
new file mode 100644
index 0000000..cdc3e6b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/cc-main.conf
@@ -0,0 +1,63 @@
+; Licensed to the Apache Software Foundation (ASF) under one
+; or more contributor license agreements.  See the NOTICE file
+; distributed with this work for additional information
+; regarding copyright ownership.  The ASF licenses this file
+; to you under the Apache License, Version 2.0 (the
+; "License"); you may not use this file except in compliance
+; with the License.  You may obtain a copy of the License at
+;
+;   http://www.apache.org/licenses/LICENSE-2.0
+;
+; Unless required by applicable law or agreed to in writing,
+; software distributed under the License is distributed on an
+; "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+; KIND, either express or implied.  See the License for the
+; specific language governing permissions and limitations
+; under the License.
+
+[nc/asterix_nc1]
+txn.log.dir=target/tmp/asterix_nc1/txnlog
+core.dump.dir=target/tmp/asterix_nc1/coredump
+iodevices=target/tmp/asterix_nc1/iodevice1,
+iodevices=../asterix-server/target/tmp/asterix_nc1/iodevice2
+nc.api.port=19004
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5006
+
+[nc/asterix_nc2]
+ncservice.port=9091
+txn.log.dir=target/tmp/asterix_nc2/txnlog
+core.dump.dir=target/tmp/asterix_nc2/coredump
+iodevices=target/tmp/asterix_nc2/iodevice1,../asterix-server/target/tmp/asterix_nc2/iodevice2
+nc.api.port=19005
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5007
+
+[nc]
+credential.file=src/test/resources/security/passwd
+python.cmd.autolocate=true
+python.env=FOO=BAR=BAZ,BAR=BAZ
+address=127.0.0.1
+command=asterixnc
+app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
+jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
+storage.buffercache.size=128MB
+storage.memorycomponent.globalbudget=512MB
+
+[cc]
+address = 127.0.0.1
+app.class=org.apache.asterix.hyracks.bootstrap.CCApplication
+heartbeat.period=2000
+heartbeat.max.misses=25
+credential.file=src/test/resources/security/passwd
+
+[common]
+log.dir = logs/
+log.level = INFO
+compiler.framesize=32KB
+compiler.sortmemory=320KB
+compiler.groupmemory=160KB
+compiler.joinmemory=256KB
+compiler.textsearchmemory=160KB
+compiler.windowmemory=192KB
+messaging.frame.size=4096
+messaging.frame.count=512
+storage.buffercache.pagesize=32KB
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/cc-single-lookup.conf b/asterixdb/asterix-app/src/test/resources/cc-single-lookup.conf
index d59d4d5..20549a8 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-single-lookup.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-single-lookup.conf
@@ -58,6 +58,7 @@
 compiler.joinmemory=256KB
 compiler.textsearchmemory=160KB
 compiler.windowmemory=192KB
+compiler.ordered.fields=false
 compiler.batch.lookup=false
 messaging.frame.size=4096
 messaging.frame.count=512
diff --git a/asterixdb/asterix-app/src/test/resources/cc-single-partition.conf b/asterixdb/asterix-app/src/test/resources/cc-single-partition.conf
index dab765b..aeb9361 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-single-partition.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-single-partition.conf
@@ -49,6 +49,7 @@
 compiler.joinmemory=256KB
 compiler.textsearchmemory=160KB
 compiler.windowmemory=192KB
+compiler.ordered.fields=false
 messaging.frame.size=4096
 messaging.frame.count=512
 storage.buffercache.pagesize=32KB
diff --git a/asterixdb/asterix-app/src/test/resources/cc-single.conf b/asterixdb/asterix-app/src/test/resources/cc-single.conf
index a6f4e89..76b29d7 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-single.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-single.conf
@@ -45,6 +45,7 @@
 compiler.joinmemory=256KB
 compiler.textsearchmemory=160KB
 compiler.windowmemory=192KB
+compiler.ordered.fields=false
 messaging.frame.size=4096
 messaging.frame.count=512
 storage.buffercache.pagesize=32KB
diff --git a/asterixdb/asterix-app/src/test/resources/cc-small-txn-log-partition.conf b/asterixdb/asterix-app/src/test/resources/cc-small-txn-log-partition.conf
index ae91f5e..79a805b 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-small-txn-log-partition.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-small-txn-log-partition.conf
@@ -49,6 +49,7 @@
 compiler.groupmemory=160KB
 compiler.joinmemory=256KB
 compiler.windowmemory=192KB
+compiler.ordered.fields=false
 messaging.frame.size=4096
 messaging.frame.count=512
 txn.log.partitionsize=2MB
diff --git a/asterixdb/asterix-app/src/test/resources/cc-ssl.conf b/asterixdb/asterix-app/src/test/resources/cc-ssl.conf
index 547beff..086944b 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-ssl.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-ssl.conf
@@ -64,6 +64,7 @@
 compiler.joinmemory=256KB
 compiler.textsearchmemory=160KB
 compiler.windowmemory=192KB
+compiler.ordered.fields=false
 messaging.frame.size=4096
 messaging.frame.count=512
 ssl.enabled=true
diff --git a/asterixdb/asterix-app/src/test/resources/cc-storage.conf b/asterixdb/asterix-app/src/test/resources/cc-storage.conf
index 68faa17..f3a39ab 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-storage.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-storage.conf
@@ -50,6 +50,7 @@
 compiler.joinmemory=256KB
 compiler.textsearchmemory=160KB
 compiler.windowmemory=192KB
+compiler.ordered.fields=false
 messaging.frame.size=4096
 messaging.frame.count=512
 storage.buffercache.pagesize=32KB
diff --git a/asterixdb/asterix-app/src/test/resources/cc.conf b/asterixdb/asterix-app/src/test/resources/cc.conf
index 624455c..075d865 100644
--- a/asterixdb/asterix-app/src/test/resources/cc.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc.conf
@@ -58,6 +58,7 @@
 compiler.joinmemory=256KB
 compiler.textsearchmemory=160KB
 compiler.windowmemory=192KB
+compiler.ordered.fields=false
 messaging.frame.size=4096
 messaging.frame.count=512
 storage.buffercache.pagesize=32KB
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-4.sqlpp
new file mode 100644
index 0000000..35ca9f9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-4.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Skip secondary index for LIKE operator
+ * Expected Res : Success
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+create collection c primary key (id: int);
+create index idx1 on c(f:string);
+select * from c where f /*+skip-index*/ like "abc%";
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 80caf74..ad746cf 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
@@ -29,154 +29,145 @@
                 -- INSERT_DELETE  |PARTITIONED|
                   -- HASH_PARTITION_EXCHANGE [$$214]  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
                         -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$267]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$267]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$267(ASC)]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$267]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                  -- STABLE_SORT [$$267(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$267]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$266][$$237]  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$266][$$237]  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- UNNEST  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- PRE_CLUSTERED_GROUP_BY[$$322]  |PARTITIONED|
-                                                                  {
-                                                                    -- AGGREGATE  |LOCAL|
-                                                                      -- MICRO_PRE_CLUSTERED_GROUP_BY[$$324, $$325]  |LOCAL|
-                                                                              {
-                                                                                -- AGGREGATE  |LOCAL|
-                                                                                  -- STREAM_SELECT  |LOCAL|
-                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                              }
-                                                                        -- STREAM_SELECT  |LOCAL|
-                                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                  }
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- STABLE_SORT [$$322(ASC), $$324(ASC), $$325(ASC)]  |PARTITIONED|
-                                                                -- HASH_PARTITION_EXCHANGE [$$322]  |PARTITIONED|
-                                                                  -- UNION_ALL  |PARTITIONED|
+                                              -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                -- UNNEST  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- PRE_CLUSTERED_GROUP_BY[$$322]  |PARTITIONED|
+                                                            {
+                                                              -- AGGREGATE  |LOCAL|
+                                                                -- MICRO_PRE_CLUSTERED_GROUP_BY[$$324, $$325]  |LOCAL|
+                                                                        {
+                                                                          -- AGGREGATE  |LOCAL|
+                                                                            -- STREAM_SELECT  |LOCAL|
+                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                        }
+                                                                  -- STREAM_SELECT  |LOCAL|
+                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                            }
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STABLE_SORT [$$322(ASC), $$324(ASC), $$325(ASC)]  |PARTITIONED|
+                                                          -- HASH_PARTITION_EXCHANGE [$$322]  |PARTITIONED|
+                                                            -- UNION_ALL  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                      -- BTREE_SEARCH (channels.Shelters.Shelters)  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                           -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- BTREE_SEARCH (channels.Shelters.Shelters)  |PARTITIONED|
-                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- SPLIT  |PARTITIONED|
-                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                -- RTREE_SEARCH (channels.Shelters.s_location)  |PARTITIONED|
-                                                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                        -- NESTED_LOOP  |PARTITIONED|
-                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                            -- NESTED_LOOP  |PARTITIONED|
-                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                    -- DATASOURCE_SCAN (channels.EmergenciesNearMeChannelChannelSubscriptions)  |PARTITIONED|
-                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                                                  -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                        -- BTREE_SEARCH (channels.Reports.Reports)  |PARTITIONED|
-                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                            -- STABLE_SORT [$$274(ASC)]  |PARTITIONED|
-                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                    -- BTREE_SEARCH (channels.Reports.report_time)  |PARTITIONED|
-                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                        -- ASSIGN  |PARTITIONED|
-                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                            -- ASSIGN  |PARTITIONED|
-                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                -- DATASOURCE_SCAN (channels.UserLocations)  |PARTITIONED|
-                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        -- STREAM_SELECT  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- SPLIT  |PARTITIONED|
                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- SPLIT  |PARTITIONED|
+                                                                                  -- STREAM_PROJECT  |PARTITIONED|
                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                          -- RTREE_SEARCH (channels.Shelters.s_location)  |PARTITIONED|
-                                                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                      -- RTREE_SEARCH (channels.Shelters.s_location)  |PARTITIONED|
+                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- NESTED_LOOP  |PARTITIONED|
                                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                   -- NESTED_LOOP  |PARTITIONED|
                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                      -- NESTED_LOOP  |PARTITIONED|
-                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          -- ASSIGN  |PARTITIONED|
-                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                              -- DATASOURCE_SCAN (channels.EmergenciesNearMeChannelChannelSubscriptions)  |PARTITIONED|
-                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                          -- ASSIGN  |PARTITIONED|
-                                                                                                            -- STREAM_SELECT  |PARTITIONED|
-                                                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                  -- BTREE_SEARCH (channels.Reports.Reports)  |PARTITIONED|
-                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                      -- STABLE_SORT [$$274(ASC)]  |PARTITIONED|
-                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                              -- BTREE_SEARCH (channels.Reports.report_time)  |PARTITIONED|
-                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
                                                                                                       -- ASSIGN  |PARTITIONED|
                                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          -- DATASOURCE_SCAN (channels.UserLocations)  |PARTITIONED|
+                                                                                                          -- DATASOURCE_SCAN (channels.EmergenciesNearMeChannelChannelSubscriptions)  |PARTITIONED|
                                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$237]  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- HYBRID_HASH_JOIN [$$248, $$250][$$239, $$240]  |PARTITIONED|
-                                                          -- HASH_PARTITION_EXCHANGE [$$248, $$250]  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (channels.EmergenciesNearMeChannelBrokerSubscriptions)  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                          -- HASH_PARTITION_EXCHANGE [$$239, $$240]  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (channels.Broker)  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- BTREE_SEARCH (channels.Reports.Reports)  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- STABLE_SORT [$$274(ASC)]  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- BTREE_SEARCH (channels.Reports.report_time)  |PARTITIONED|
+                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                      -- DATASOURCE_SCAN (channels.UserLocations)  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- SPLIT  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- RTREE_SEARCH (channels.Shelters.s_location)  |PARTITIONED|
+                                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- NESTED_LOOP  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- NESTED_LOOP  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- DATASOURCE_SCAN (channels.EmergenciesNearMeChannelChannelSubscriptions)  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- BTREE_SEARCH (channels.Reports.Reports)  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                            -- STABLE_SORT [$$274(ASC)]  |PARTITIONED|
+                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                    -- BTREE_SEARCH (channels.Reports.report_time)  |PARTITIONED|
+                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- DATASOURCE_SCAN (channels.UserLocations)  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$237]  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$248, $$250][$$239, $$240]  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$248, $$250]  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (channels.EmergenciesNearMeChannelBrokerSubscriptions)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$239, $$240]  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (channels.Broker)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.10.plan
index 21bd922..665d725 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.10.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.10.plan
@@ -1,35 +1,33 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                        -- MICRO_STABLE_SORT [$$67(ASC)]  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
-                  -- NESTED_LOOP  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- NESTED_LOOP  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                      -- MICRO_STABLE_SORT [$$72(ASC)]  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- NESTED_LOOP  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                      -- UNNEST  |UNPARTITIONED|
-                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- UNNEST  |UNPARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.11.plan
index 0e17c71..55a8c3a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.11.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.11.plan
@@ -1,43 +1,40 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                        -- MICRO_STABLE_SORT [$$67(ASC)]  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                        -- MICRO_STABLE_SORT [$$72(ASC)]  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
-                  -- NESTED_LOOP  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- NESTED_LOOP  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                      -- MICRO_STABLE_SORT [$$72(ASC)]  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                      -- MICRO_STABLE_SORT [$$77(ASC)]  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- NESTED_LOOP  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                      -- UNNEST  |UNPARTITIONED|
-                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- UNNEST  |UNPARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.12.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.12.plan
index 746b781..523e336 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.12.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.12.plan
@@ -1,39 +1,37 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                        -- MICRO_STABLE_SORT [$$71(ASC)]  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
-                  -- NESTED_LOOP  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- NESTED_LOOP  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                      -- MICRO_STABLE_SORT [$$76(ASC)]  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- NESTED_LOOP  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                      -- UNNEST  |UNPARTITIONED|
-                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- UNNEST  |UNPARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.13.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.13.plan
index 60d2ed2..aefdb9a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.13.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.13.plan
@@ -1,57 +1,53 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                        -- MICRO_STABLE_SORT [$$82(ASC)]  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                        -- MICRO_STABLE_SORT [$$92(ASC)]  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
-                  -- NESTED_LOOP  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- NESTED_LOOP  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- NESTED_LOOP  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.d1)  |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|
-                                      -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                      -- MICRO_STABLE_SORT [$$88(ASC)]  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                      -- MICRO_STABLE_SORT [$$98(ASC)]  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- NESTED_LOOP  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- NESTED_LOOP  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.d3)  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                      -- UNNEST  |UNPARTITIONED|
-                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.d3)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- UNNEST  |UNPARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.14.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.14.plan
index 247fc31..c522bc1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.14.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.14.plan
@@ -1,63 +1,59 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                        -- MICRO_STABLE_SORT [$$106(ASC)]  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                        -- MICRO_STABLE_SORT [$$116(ASC)]  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                        -- MICRO_STABLE_SORT [$$111(ASC)]  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
-                  -- NESTED_LOOP  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- NESTED_LOOP  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- NESTED_LOOP  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.d1)  |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|
-                                      -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                      -- MICRO_STABLE_SORT [$$112(ASC)]  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                      -- MICRO_STABLE_SORT [$$122(ASC)]  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                      -- MICRO_STABLE_SORT [$$117(ASC)]  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- NESTED_LOOP  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- NESTED_LOOP  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.d3)  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                      -- UNNEST  |UNPARTITIONED|
-                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.d3)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- UNNEST  |UNPARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.15.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.15.plan
index 5c97374..5485ea5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.15.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.15.plan
@@ -1,175 +1,162 @@
 -- DISTRIBUTE_RESULT  |LOCAL|
   -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-    -- STREAM_PROJECT  |LOCAL|
-      -- ASSIGN  |LOCAL|
-        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-          -- NESTED_LOOP  |LOCAL|
-            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-              -- NESTED_LOOP  |LOCAL|
-                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                  -- NESTED_LOOP  |LOCAL|
-                    -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                      -- AGGREGATE  |LOCAL|
-                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                          -- PRE_SORTED_DISTINCT_BY  |LOCAL|
-                            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                              -- STABLE_SORT [$$133(ASC)]  |LOCAL|
-                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                  -- NESTED_LOOP  |UNPARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                      -- NESTED_LOOP  |UNPARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                          -- REPLICATE  |UNPARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                              -- REPLICATE  |UNPARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                  -- UNNEST  |UNPARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                          -- REPLICATE  |UNPARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                              -- STREAM_PROJECT  |UNPARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                  -- REPLICATE  |UNPARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                      -- UNNEST  |UNPARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                      -- REPLICATE  |UNPARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                          -- STREAM_PROJECT  |UNPARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                              -- REPLICATE  |UNPARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                  -- UNNEST  |UNPARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                      -- AGGREGATE  |LOCAL|
-                        -- AGGREGATE  |LOCAL|
-                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                            -- NESTED_LOOP  |UNPARTITIONED|
+    -- ASSIGN  |LOCAL|
+      -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+        -- NESTED_LOOP  |LOCAL|
+          -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+            -- NESTED_LOOP  |LOCAL|
+              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                -- NESTED_LOOP  |LOCAL|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- AGGREGATE  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                        -- PRE_SORTED_DISTINCT_BY  |LOCAL|
+                          -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                            -- STABLE_SORT [$$141(ASC)]  |LOCAL|
                               -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
                                 -- NESTED_LOOP  |UNPARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                    -- STREAM_PROJECT  |UNPARTITIONED|
-                                      -- ASSIGN  |UNPARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                          -- REPLICATE  |UNPARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                              -- REPLICATE  |UNPARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                  -- UNNEST  |UNPARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                    -- REPLICATE  |UNPARTITIONED|
+                                    -- NESTED_LOOP  |UNPARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                        -- STREAM_PROJECT  |UNPARTITIONED|
-                                          -- ASSIGN  |UNPARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                              -- REPLICATE  |UNPARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                  -- UNNEST  |UNPARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                -- STREAM_PROJECT  |UNPARTITIONED|
-                                  -- ASSIGN  |UNPARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                      -- REPLICATE  |UNPARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                          -- STREAM_PROJECT  |UNPARTITIONED|
-                                            -- ASSIGN  |UNPARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                -- REPLICATE  |UNPARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                    -- UNNEST  |UNPARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                  -- AGGREGATE  |LOCAL|
-                    -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                      -- PRE_SORTED_DISTINCT_BY  |LOCAL|
-                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                          -- STABLE_SORT [$$159(ASC)]  |LOCAL|
-                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                              -- NESTED_LOOP  |UNPARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                  -- NESTED_LOOP  |UNPARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                      -- REPLICATE  |UNPARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                          -- STREAM_PROJECT  |UNPARTITIONED|
-                                            -- ASSIGN  |UNPARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                -- REPLICATE  |UNPARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                    -- UNNEST  |UNPARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                      -- STREAM_PROJECT  |UNPARTITIONED|
-                                        -- ASSIGN  |UNPARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                            -- REPLICATE  |UNPARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                -- STREAM_PROJECT  |UNPARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                    -- REPLICATE  |UNPARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                        -- UNNEST  |UNPARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                  -- REPLICATE  |UNPARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                      -- STREAM_PROJECT  |UNPARTITIONED|
-                                        -- ASSIGN  |UNPARTITIONED|
+                                        -- REPLICATE  |UNPARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
                                             -- REPLICATE  |UNPARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
                                                 -- UNNEST  |UNPARTITIONED|
                                                   -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-              -- AGGREGATE  |LOCAL|
-                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                  -- PRE_SORTED_DISTINCT_BY  |LOCAL|
-                    -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                      -- STABLE_SORT [$$y(ASC)]  |LOCAL|
+                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                        -- REPLICATE  |UNPARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                            -- STREAM_PROJECT  |UNPARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                -- REPLICATE  |UNPARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                    -- UNNEST  |UNPARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                    -- REPLICATE  |UNPARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                        -- STREAM_PROJECT  |UNPARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                            -- REPLICATE  |UNPARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- AGGREGATE  |LOCAL|
+                      -- AGGREGATE  |LOCAL|
                         -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
                           -- NESTED_LOOP  |UNPARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
                               -- NESTED_LOOP  |UNPARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                  -- STREAM_PROJECT  |UNPARTITIONED|
-                                    -- ASSIGN  |UNPARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                        -- REPLICATE  |UNPARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                            -- STREAM_PROJECT  |UNPARTITIONED|
-                                              -- ASSIGN  |UNPARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                  -- REPLICATE  |UNPARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                      -- UNNEST  |UNPARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                  -- ASSIGN  |UNPARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                      -- REPLICATE  |UNPARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                          -- REPLICATE  |UNPARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                              -- UNNEST  |UNPARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                  -- STREAM_PROJECT  |UNPARTITIONED|
-                                    -- ASSIGN  |UNPARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                        -- REPLICATE  |UNPARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                            -- STREAM_PROJECT  |UNPARTITIONED|
-                                              -- ASSIGN  |UNPARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                  -- REPLICATE  |UNPARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                      -- UNNEST  |UNPARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                  -- REPLICATE  |UNPARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                      -- ASSIGN  |UNPARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                          -- REPLICATE  |UNPARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                              -- UNNEST  |UNPARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                              -- STREAM_PROJECT  |UNPARTITIONED|
-                                -- ASSIGN  |UNPARTITIONED|
+                              -- ASSIGN  |UNPARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                  -- REPLICATE  |UNPARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                      -- ASSIGN  |UNPARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                          -- REPLICATE  |UNPARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                              -- UNNEST  |UNPARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                -- AGGREGATE  |LOCAL|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- PRE_SORTED_DISTINCT_BY  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                        -- STABLE_SORT [$$167(ASC)]  |LOCAL|
+                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                            -- NESTED_LOOP  |UNPARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                -- NESTED_LOOP  |UNPARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
                                     -- REPLICATE  |UNPARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                        -- STREAM_PROJECT  |UNPARTITIONED|
+                                        -- ASSIGN  |UNPARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
                                             -- REPLICATE  |UNPARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
                                                 -- UNNEST  |UNPARTITIONED|
                                                   -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                    -- ASSIGN  |UNPARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                        -- REPLICATE  |UNPARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                            -- STREAM_PROJECT  |UNPARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                -- REPLICATE  |UNPARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                    -- UNNEST  |UNPARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                -- REPLICATE  |UNPARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                    -- ASSIGN  |UNPARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                        -- REPLICATE  |UNPARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                            -- UNNEST  |UNPARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+            -- AGGREGATE  |LOCAL|
+              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                -- PRE_SORTED_DISTINCT_BY  |LOCAL|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$y(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                        -- NESTED_LOOP  |UNPARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                            -- NESTED_LOOP  |UNPARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                -- ASSIGN  |UNPARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                    -- REPLICATE  |UNPARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                        -- ASSIGN  |UNPARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                            -- REPLICATE  |UNPARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                -- ASSIGN  |UNPARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                    -- REPLICATE  |UNPARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                        -- ASSIGN  |UNPARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                            -- REPLICATE  |UNPARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                            -- ASSIGN  |UNPARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                -- REPLICATE  |UNPARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                    -- STREAM_PROJECT  |UNPARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                        -- REPLICATE  |UNPARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                            -- UNNEST  |UNPARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.16.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.16.plan
index 79505b4..481e9fe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.16.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.16.plan
@@ -1,49 +1,48 @@
 -- DISTRIBUTE_RESULT  |LOCAL|
   -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-    -- STREAM_PROJECT  |LOCAL|
-      -- ASSIGN  |LOCAL|
-        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-          -- PRE_CLUSTERED_GROUP_BY[$$g]  |LOCAL|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                        -- MICRO_STABLE_SORT [$$x(ASC)]  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- AGGREGATE  |LOCAL|
+    -- ASSIGN  |LOCAL|
+      -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+        -- PRE_CLUSTERED_GROUP_BY[$$g]  |LOCAL|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                      -- MICRO_STABLE_SORT [$$x(ASC)]  |LOCAL|
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-                  {
+                }
+                {
+                  -- AGGREGATE  |LOCAL|
                     -- AGGREGATE  |LOCAL|
-                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                        -- MICRO_STABLE_SORT [$$z(ASC)]  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                        -- MICRO_STABLE_SORT [$$y(ASC)]  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-              -- STABLE_SORT [$$g(ASC)]  |LOCAL|
-                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                  -- NESTED_LOOP  |UNPARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                      -- NESTED_LOOP  |UNPARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                          -- NESTED_LOOP  |UNPARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                              -- UNNEST  |UNPARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                              -- UNNEST  |UNPARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                          -- UNNEST  |UNPARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                      -- UNNEST  |UNPARTITIONED|
-                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                      -- MICRO_STABLE_SORT [$$z(ASC)]  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                      -- MICRO_STABLE_SORT [$$y(ASC)]  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+            -- STABLE_SORT [$$g(ASC)]  |LOCAL|
+              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                -- NESTED_LOOP  |UNPARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                    -- NESTED_LOOP  |UNPARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                        -- NESTED_LOOP  |UNPARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                            -- UNNEST  |UNPARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                            -- UNNEST  |UNPARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                        -- UNNEST  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                    -- UNNEST  |UNPARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.17.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.17.plan
index 1821d02..cc28b55 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.17.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.17.plan
@@ -2,93 +2,85 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- UNION_ALL  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- PRE_CLUSTERED_GROUP_BY[$$156]  |PARTITIONED|
-                        {
-                          -- AGGREGATE  |LOCAL|
-                            -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                              -- MICRO_STABLE_SORT [$$124(ASC)]  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                        }
-                        {
-                          -- AGGREGATE  |LOCAL|
-                            -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                              -- MICRO_STABLE_SORT [$$129(ASC)]  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                        }
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$156(ASC)]  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$156]  |PARTITIONED|
-                        -- NESTED_LOOP  |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|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.d1)  |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|
-                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- PRE_CLUSTERED_GROUP_BY[$$157]  |PARTITIONED|
-                        {
-                          -- AGGREGATE  |LOCAL|
-                            -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                              -- MICRO_STABLE_SORT [$$142(ASC)]  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                        }
-                        {
-                          -- AGGREGATE  |LOCAL|
-                            -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                              -- MICRO_STABLE_SORT [$$147(ASC)]  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                        }
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$157(ASC)]  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$157]  |PARTITIONED|
-                        -- NESTED_LOOP  |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 (test.d1)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- PRE_CLUSTERED_GROUP_BY[$$156]  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                            -- MICRO_STABLE_SORT [$$124(ASC)]  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                            -- MICRO_STABLE_SORT [$$129(ASC)]  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$156(ASC)]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$156]  |PARTITIONED|
+                      -- NESTED_LOOP  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |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 (test.d1)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- PRE_CLUSTERED_GROUP_BY[$$157]  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                            -- MICRO_STABLE_SORT [$$142(ASC)]  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                            -- MICRO_STABLE_SORT [$$147(ASC)]  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$157(ASC)]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$157]  |PARTITIONED|
+                      -- NESTED_LOOP  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |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 (test.d1)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.3.plan
index 38f23fd..704690c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.3.plan
@@ -1,25 +1,23 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$$42(ASC) ]  |PARTITIONED|
-            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
-                    -- NESTED_LOOP  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$45(ASC) ]  |PARTITIONED|
+          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$45]  |PARTITIONED|
+                  -- NESTED_LOOP  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.4.plan
index 4439523..f48a2c6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.4.plan
@@ -1,25 +1,23 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$$46(ASC) ]  |PARTITIONED|
-            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$46]  |PARTITIONED|
-                    -- NESTED_LOOP  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$49(ASC) ]  |PARTITIONED|
+          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$49(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$49]  |PARTITIONED|
+                  -- NESTED_LOOP  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.5.plan
index 1ee89a8..f555a00 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.5.plan
@@ -1,60 +1,55 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-          -- NESTED_LOOP  |UNPARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-              -- AGGREGATE  |UNPARTITIONED|
-                -- SORT_MERGE_EXCHANGE [$$62(ASC) ]  |PARTITIONED|
-                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$62(ASC)]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
-                          -- NESTED_LOOP  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+        -- NESTED_LOOP  |UNPARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+            -- AGGREGATE  |UNPARTITIONED|
+              -- SORT_MERGE_EXCHANGE [$$65(ASC) ]  |PARTITIONED|
+                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$65(ASC)]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$65]  |PARTITIONED|
+                        -- NESTED_LOOP  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- REPLICATE  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- REPLICATE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- REPLICATE  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- REPLICATE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-              -- AGGREGATE  |UNPARTITIONED|
-                -- SORT_MERGE_EXCHANGE [$$51(ASC) ]  |PARTITIONED|
-                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$51(ASC)]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$51]  |PARTITIONED|
-                          -- NESTED_LOOP  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+            -- AGGREGATE  |UNPARTITIONED|
+              -- SORT_MERGE_EXCHANGE [$$54(ASC) ]  |PARTITIONED|
+                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+                        -- NESTED_LOOP  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- REPLICATE  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- REPLICATE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- REPLICATE  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- REPLICATE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.6.plan
index 6c7bcd3..6fd488e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.6.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.6.plan
@@ -1,62 +1,57 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-          -- NESTED_LOOP  |UNPARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-              -- AGGREGATE  |UNPARTITIONED|
-                -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- NESTED_LOOP  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+        -- NESTED_LOOP  |UNPARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+            -- AGGREGATE  |UNPARTITIONED|
+              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- NESTED_LOOP  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- REPLICATE  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (test.d1)  |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|
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-              -- AGGREGATE  |UNPARTITIONED|
-                -- SORT_MERGE_EXCHANGE [$$50(ASC) ]  |PARTITIONED|
-                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$50]  |PARTITIONED|
-                          -- NESTED_LOOP  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
-                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+            -- AGGREGATE  |UNPARTITIONED|
+              -- SORT_MERGE_EXCHANGE [$$53(ASC) ]  |PARTITIONED|
+                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$53(ASC)]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                        -- NESTED_LOOP  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.7.plan
index 274a43a..fcda7e7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.7.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.7.plan
@@ -1,86 +1,111 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-          -- NESTED_LOOP  |UNPARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-              -- NESTED_LOOP  |UNPARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- SORT_MERGE_EXCHANGE [$$86(ASC) ]  |PARTITIONED|
-                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$86(ASC)]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$86]  |PARTITIONED|
-                              -- NESTED_LOOP  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- NESTED_LOOP  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- REPLICATE  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- REPLICATE  |PARTITIONED|
-                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- REPLICATE  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- REPLICATE  |PARTITIONED|
-                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+        -- NESTED_LOOP  |UNPARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+            -- NESTED_LOOP  |UNPARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- SORT_MERGE_EXCHANGE [$$90(ASC) ]  |PARTITIONED|
+                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$90(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$90]  |PARTITIONED|
+                            -- NESTED_LOOP  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- NESTED_LOOP  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- REPLICATE  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.d3)  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- NESTED_LOOP  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- NESTED_LOOP  |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|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- REPLICATE  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- REPLICATE  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                 -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- REPLICATE  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN (test.d3)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- NESTED_LOOP  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- NESTED_LOOP  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- REPLICATE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- REPLICATE  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- REPLICATE  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN (test.d3)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+            -- AGGREGATE  |UNPARTITIONED|
+              -- SORT_MERGE_EXCHANGE [$$75(ASC) ]  |PARTITIONED|
+                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$75(ASC)]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$75]  |PARTITIONED|
+                        -- NESTED_LOOP  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- NESTED_LOOP  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- REPLICATE  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- ASSIGN  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- REPLICATE  |PARTITIONED|
@@ -89,51 +114,15 @@
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- REPLICATE  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN (test.d3)  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-              -- AGGREGATE  |UNPARTITIONED|
-                -- SORT_MERGE_EXCHANGE [$$71(ASC) ]  |PARTITIONED|
-                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$71(ASC)]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$71]  |PARTITIONED|
-                          -- NESTED_LOOP  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- NESTED_LOOP  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |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|
-                                                -- DATASOURCE_SCAN (test.d1)  |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|
-                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- REPLICATE  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- REPLICATE  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (test.d3)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.d3)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.8.plan
index a6f3df3..91e3ad3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.8.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.8.plan
@@ -1,204 +1,185 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-          -- NESTED_LOOP  |UNPARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-              -- NESTED_LOOP  |UNPARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                  -- NESTED_LOOP  |UNPARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                      -- AGGREGATE  |UNPARTITIONED|
-                        -- SORT_MERGE_EXCHANGE [$$152(ASC) ]  |PARTITIONED|
-                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$152(ASC)]  |PARTITIONED|
-                                -- HASH_PARTITION_EXCHANGE [$$152]  |PARTITIONED|
-                                  -- NESTED_LOOP  |PARTITIONED|
-                                    -- 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|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- REPLICATE  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- REPLICATE  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- REPLICATE  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- REPLICATE  |PARTITIONED|
-                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- REPLICATE  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (test.d3)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                      -- AGGREGATE  |UNPARTITIONED|
-                        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                          -- AGGREGATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- NESTED_LOOP  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- NESTED_LOOP  |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|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- REPLICATE  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- REPLICATE  |PARTITIONED|
-                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- REPLICATE  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (test.d2)  |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|
-                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- REPLICATE  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (test.d3)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- SORT_MERGE_EXCHANGE [$$190(ASC) ]  |PARTITIONED|
-                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$190(ASC)]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$190]  |PARTITIONED|
-                              -- NESTED_LOOP  |PARTITIONED|
-                                -- 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|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- REPLICATE  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (test.d1)  |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|
-                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- REPLICATE  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- REPLICATE  |PARTITIONED|
-                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- REPLICATE  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- DATASOURCE_SCAN (test.d3)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-              -- AGGREGATE  |UNPARTITIONED|
-                -- SORT_MERGE_EXCHANGE [$$90(ASC) ]  |PARTITIONED|
-                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$90(ASC)]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$90]  |PARTITIONED|
-                          -- NESTED_LOOP  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- NESTED_LOOP  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+        -- NESTED_LOOP  |UNPARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+            -- NESTED_LOOP  |UNPARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                -- NESTED_LOOP  |UNPARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- SORT_MERGE_EXCHANGE [$$156(ASC) ]  |PARTITIONED|
+                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$156(ASC)]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$156]  |PARTITIONED|
+                                -- NESTED_LOOP  |PARTITIONED|
+                                  -- 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|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                   -- REPLICATE  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                       -- DATASOURCE_SCAN (test.d1)  |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|
                                           -- BROADCAST_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|
+                                                    -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- REPLICATE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.d3)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- NESTED_LOOP  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- NESTED_LOOP  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- REPLICATE  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- REPLICATE  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- ASSIGN  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- REPLICATE  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                               -- REPLICATE  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                   -- DATASOURCE_SCAN (test.d3)  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                       -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- SORT_MERGE_EXCHANGE [$$194(ASC) ]  |PARTITIONED|
+                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$194(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$194]  |PARTITIONED|
+                            -- NESTED_LOOP  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- NESTED_LOOP  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- REPLICATE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- REPLICATE  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- REPLICATE  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- REPLICATE  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.d3)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+            -- AGGREGATE  |UNPARTITIONED|
+              -- SORT_MERGE_EXCHANGE [$$94(ASC) ]  |PARTITIONED|
+                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$94(ASC)]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$94]  |PARTITIONED|
+                        -- NESTED_LOOP  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- NESTED_LOOP  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- REPLICATE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- REPLICATE  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- REPLICATE  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN (test.d3)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.9.plan
index 76735bd..979d032 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.9.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-sql-sugar/distinct_mixed/distinct_mixed.9.plan
@@ -1,35 +1,33 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                        -- MICRO_STABLE_SORT [$$63(ASC)]  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
-                  -- NESTED_LOOP  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- NESTED_LOOP  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                      -- MICRO_STABLE_SORT [$$68(ASC)]  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- NESTED_LOOP  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                      -- UNNEST  |UNPARTITIONED|
-                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- UNNEST  |UNPARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.10.plan
index 63d3f7c..9887601 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.10.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.10.plan
@@ -1,27 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$ten(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$95]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$95]  |PARTITIONED|
-              -- PRE_CLUSTERED_GROUP_BY[$$82]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$82(ASC)]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$ten(ASC) ]  |PARTITIONED|
+        -- SORT_GROUP_BY[$$98]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$98]  |PARTITIONED|
+            -- PRE_CLUSTERED_GROUP_BY[$$85]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- STREAM_SELECT  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$85(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (test.tenk)  |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/aggregate-subclause/agg_filter_01/agg_filter_01.3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.3.plan
index 574df58..52403ee 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.3.plan
@@ -1,16 +1,14 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.4.plan
index c94ef11..a45aec6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.4.plan
@@ -1,27 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$two(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$67]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$67]  |PARTITIONED|
-              -- PRE_CLUSTERED_GROUP_BY[$$58]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$two(ASC) ]  |PARTITIONED|
+        -- SORT_GROUP_BY[$$70]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$70]  |PARTITIONED|
+            -- PRE_CLUSTERED_GROUP_BY[$$61]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- STREAM_SELECT  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.5.plan
index 9462514..28d7ebb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.5.plan
@@ -1,68 +1,63 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$189(ASC) ]  |PARTITIONED|
-        -- STABLE_SORT [$$189(ASC)]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$192(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$192(ASC)]  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- UNION_ALL  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- SORT_GROUP_BY[$$239]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- HASH_PARTITION_EXCHANGE [$$239]  |PARTITIONED|
+                        -- PRE_CLUSTERED_GROUP_BY[$$118]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$118(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |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 (test.tenk)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- SORT_GROUP_BY[$$236]  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$242]  |PARTITIONED|
                               {
                                 -- AGGREGATE  |LOCAL|
                                   -- NESTED_TUPLE_SOURCE  |LOCAL|
                               }
-                        -- HASH_PARTITION_EXCHANGE [$$236]  |PARTITIONED|
-                          -- PRE_CLUSTERED_GROUP_BY[$$115]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$242]  |PARTITIONED|
+                          -- PRE_CLUSTERED_GROUP_BY[$$119]  |PARTITIONED|
                                   {
                                     -- AGGREGATE  |LOCAL|
                                       -- STREAM_SELECT  |LOCAL|
                                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                                   }
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$115(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$119(ASC)]  |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|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- SORT_GROUP_BY[$$239]  |PARTITIONED|
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
-                          -- HASH_PARTITION_EXCHANGE [$$239]  |PARTITIONED|
-                            -- PRE_CLUSTERED_GROUP_BY[$$116]  |PARTITIONED|
-                                    {
-                                      -- AGGREGATE  |LOCAL|
-                                        -- STREAM_SELECT  |LOCAL|
-                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                    }
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$116(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |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 (test.tenk)  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.6.plan
index 3f5657c..5692b93 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.6.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.6.plan
@@ -1,26 +1,24 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- WINDOW  |UNPARTITIONED|
+              {
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- STREAM_SELECT  |UNPARTITIONED|
+                    -- NESTED_TUPLE_SOURCE  |UNPARTITIONED|
+              }
         -- WINDOW  |UNPARTITIONED|
                 {
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- STREAM_SELECT  |UNPARTITIONED|
-                      -- NESTED_TUPLE_SOURCE  |UNPARTITIONED|
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
                 }
-          -- WINDOW  |UNPARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- SORT_MERGE_EXCHANGE [$$tenk.tenthous(ASC) ]  |PARTITIONED|
-              -- STABLE_SORT [$$tenk.tenthous(ASC)]  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$tenk.tenthous(ASC) ]  |PARTITIONED|
+            -- STABLE_SORT [$$tenk.tenthous(ASC)]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.7.plan
index 900d1b5..1b586cb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.7.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.7.plan
@@ -1,44 +1,42 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- STREAM_PROJECT  |UNPARTITIONED|
-          -- WINDOW  |UNPARTITIONED|
-                  {
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- STREAM_SELECT  |UNPARTITIONED|
-                        -- NESTED_TUPLE_SOURCE  |UNPARTITIONED|
-                  }
-            -- STREAM_PROJECT  |UNPARTITIONED|
-              -- WINDOW  |UNPARTITIONED|
-                      {
-                        -- AGGREGATE  |UNPARTITIONED|
-                          -- STREAM_SELECT  |UNPARTITIONED|
-                            -- NESTED_TUPLE_SOURCE  |UNPARTITIONED|
-                      }
-                -- STREAM_PROJECT  |UNPARTITIONED|
-                  -- WINDOW  |UNPARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- SORT_MERGE_EXCHANGE [$$twenty(ASC) ]  |PARTITIONED|
-                      -- SORT_GROUP_BY[$$201]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- HASH_PARTITION_EXCHANGE [$$201]  |PARTITIONED|
-                          -- SORT_GROUP_BY[$$183]  |PARTITIONED|
-                                  {
-                                    -- AGGREGATE  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                  }
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- STREAM_PROJECT  |UNPARTITIONED|
+        -- WINDOW  |UNPARTITIONED|
+                {
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- STREAM_SELECT  |UNPARTITIONED|
+                      -- NESTED_TUPLE_SOURCE  |UNPARTITIONED|
+                }
+          -- STREAM_PROJECT  |UNPARTITIONED|
+            -- WINDOW  |UNPARTITIONED|
+                    {
+                      -- AGGREGATE  |UNPARTITIONED|
+                        -- STREAM_SELECT  |UNPARTITIONED|
+                          -- NESTED_TUPLE_SOURCE  |UNPARTITIONED|
+                    }
+              -- STREAM_PROJECT  |UNPARTITIONED|
+                -- WINDOW  |UNPARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- SORT_MERGE_EXCHANGE [$$twenty(ASC) ]  |PARTITIONED|
+                    -- SORT_GROUP_BY[$$201]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- HASH_PARTITION_EXCHANGE [$$201]  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$183]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.8.plan
index 4ceb881..fd9f30c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.8.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.8.plan
@@ -1,28 +1,26 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$two(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$66]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- STREAM_SELECT  |LOCAL|
-                        -- SUBPLAN  |LOCAL|
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- STREAM_SELECT  |LOCAL|
-                                      -- UNNEST  |LOCAL|
-                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$66(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$66]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$two(ASC) ]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$70]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- STREAM_SELECT  |LOCAL|
+                      -- SUBPLAN  |LOCAL|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- UNNEST  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$70(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$70]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.9.plan
index 02d8027..52403ee 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.9.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.9.plan
@@ -1,16 +1,14 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.tenk)  |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/aggregate/constant-agg.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/constant-agg.plan
index b79b733..33300f5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/constant-agg.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/constant-agg.plan
@@ -1,11 +1,10 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- DATASOURCE_SCAN (TinySocial.FacebookUsers)  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN (TinySocial.FacebookUsers)  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/constant-gby-agg.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/constant-gby-agg.plan
index 453e53b..fd2f29d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/constant-gby-agg.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/constant-gby-agg.plan
@@ -1,24 +1,22 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- SORT_GROUP_BY[$$47]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$47]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$43]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- SORT_GROUP_BY[$$50]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$50]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$46]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |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|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/query-ASTERIXDB-1626-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/query-ASTERIXDB-1626-2.plan
index 5304d60..46a1291 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/query-ASTERIXDB-1626-2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/query-ASTERIXDB-1626-2.plan
@@ -1,15 +1,13 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |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|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/query-ASTERIXDB-1626-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/query-ASTERIXDB-1626-3.plan
index 5304d60..46a1291 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/query-ASTERIXDB-1626-3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/query-ASTERIXDB-1626-3.plan
@@ -1,15 +1,13 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |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|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/query-ASTERIXDB-1626.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/query-ASTERIXDB-1626.plan
index 5304d60..46a1291 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/query-ASTERIXDB-1626.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/query-ASTERIXDB-1626.plan
@@ -1,15 +1,13 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |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|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/serial-agg.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/serial-agg.plan
index 8b7a8d7..9de2dc1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/serial-agg.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/serial-agg.plan
@@ -1,24 +1,22 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- EXTERNAL_GROUP_BY[$$50]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$50]  |PARTITIONED|
-              -- EXTERNAL_GROUP_BY[$$45]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- EXTERNAL_GROUP_BY[$$53]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+            -- EXTERNAL_GROUP_BY[$$48]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.Test)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.Test)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-1.plan
index faac4b0..6efb86d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-1.plan
@@ -1,19 +1,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- UNNEST  |LOCAL|
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- UNNEST  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (TestDataverse.Dataset1)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (TestDataverse.Dataset1)  |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/any-and-every-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-2.plan
index b482afb..e85b511 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-2.plan
@@ -1,26 +1,24 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- SUBPLAN  |LOCAL|
-                                  {
-                                    -- AGGREGATE  |LOCAL|
-                                      -- UNNEST  |LOCAL|
-                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                  }
-                            -- ASSIGN  |LOCAL|
-                              -- UNNEST  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- SUBPLAN  |LOCAL|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- UNNEST  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ASSIGN  |LOCAL|
+                          -- UNNEST  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (TestDataverse.Dataset1)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (TestDataverse.Dataset1)  |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/any-and-every-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-3.plan
index d865914..aaac121 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-3.plan
@@ -1,20 +1,18 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- UNNEST  |LOCAL|
-                            -- UNNEST  |LOCAL|
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- UNNEST  |LOCAL|
+                        -- UNNEST  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (TestDataverse.Dataset1)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (TestDataverse.Dataset1)  |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/array-index/atomic-and-array-queries/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query1.plan
index d53724e..ed2e3eb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query1.plan
@@ -1,23 +1,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- UNNEST  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                          -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query2.plan
index ab9c768..4e34538 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query2.plan
@@ -1,23 +1,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- UNNEST  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                          -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query3.plan
index 2c6241a..c4fc356 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query3.plan
@@ -1,27 +1,24 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- UNNEST  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- UNNEST  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$70(ASC)]  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$73(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query4.plan
index d25efc7..ed5677d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query4.plan
@@ -1,31 +1,29 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- ASSIGN  |LOCAL|
-                              -- UNNEST  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- ASSIGN  |LOCAL|
+                          -- UNNEST  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- STABLE_SORT [$$56(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
+                                    -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query5.plan
index df8e08b..21e5109 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query5.plan
@@ -1,40 +1,38 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- SUBPLAN  |LOCAL|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- ASSIGN  |LOCAL|
+                                        -- UNNEST  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
                           -- STREAM_SELECT  |LOCAL|
-                            -- SUBPLAN  |LOCAL|
-                                    {
-                                      -- AGGREGATE  |LOCAL|
-                                        -- STREAM_SELECT  |LOCAL|
-                                          -- ASSIGN  |LOCAL|
-                                            -- UNNEST  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                    }
-                              -- STREAM_SELECT  |LOCAL|
-                                -- ASSIGN  |LOCAL|
-                                  -- UNNEST  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |LOCAL|
+                              -- UNNEST  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- STABLE_SORT [$$79(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$76(ASC)]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
+                                    -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query6.plan
index 492b219..57502ba 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query6.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query6.plan
@@ -1,33 +1,28 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STABLE_SORT [$$93(ASC), $$94(ASC)]  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
-                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                    -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- DATASOURCE_SCAN (TestDataverse.Dataset2)  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- UNNEST  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$97(ASC), $$98(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- RUNNING_AGGREGATE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (TestDataverse.Dataset2)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query7.plan
index 63544ef..65775fc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query7.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query7.plan
@@ -1,33 +1,28 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STABLE_SORT [$$111(ASC), $$112(ASC)]  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
-                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                    -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- DATASOURCE_SCAN (TestDataverse.Dataset2)  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- UNNEST  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$115(ASC), $$116(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- RUNNING_AGGREGATE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (TestDataverse.Dataset2)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query8.plan
index 3f740b0..9bf272a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query8.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query8.plan
@@ -1,38 +1,36 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- ASSIGN  |LOCAL|
-                              -- UNNEST  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- ASSIGN  |LOCAL|
+                          -- UNNEST  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                              -- STABLE_SORT [$$77(ASC), $$78(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$77(ASC), $$78(ASC)]  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                              -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- RUNNING_AGGREGATE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (TestDataverse.Dataset2)  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (TestDataverse.Dataset2)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query9.plan
index ed318b1..ab7ffda 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query9.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query9.plan
@@ -1,47 +1,45 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- SUBPLAN  |LOCAL|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- ASSIGN  |LOCAL|
+                                        -- UNNEST  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
                           -- STREAM_SELECT  |LOCAL|
-                            -- SUBPLAN  |LOCAL|
-                                    {
-                                      -- AGGREGATE  |LOCAL|
-                                        -- STREAM_SELECT  |LOCAL|
-                                          -- ASSIGN  |LOCAL|
-                                            -- UNNEST  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                    }
-                              -- STREAM_SELECT  |LOCAL|
-                                -- ASSIGN  |LOCAL|
-                                  -- UNNEST  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |LOCAL|
+                              -- UNNEST  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                              -- STABLE_SORT [$$108(ASC), $$109(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$108(ASC), $$109(ASC)]  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                              -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- RUNNING_AGGREGATE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (TestDataverse.Dataset2)  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (TestDataverse.Dataset2)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query1.plan
index 7306c4d..9526cce 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query1.plan
@@ -1,39 +1,36 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- UNNEST  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- UNNEST  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                      -- STABLE_SORT [$$64(ASC), $$56(ASC)]  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STABLE_SORT [$$61(ASC), $$53(ASC)]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                              -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query2.plan
index cb70d4f..90476c2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query2.plan
@@ -1,39 +1,36 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- UNNEST  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- UNNEST  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                      -- STABLE_SORT [$$73(ASC), $$65(ASC)]  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STABLE_SORT [$$69(ASC), $$61(ASC)]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                              -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query3.plan
index b81bf80..e2e4669 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query3.plan
@@ -1,39 +1,36 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- UNNEST  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- UNNEST  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                      -- STABLE_SORT [$$77(ASC), $$68(ASC)]  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STABLE_SORT [$$73(ASC), $$64(ASC)]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                              -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query4.plan
index d7acd37..b8e4112 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query4.plan
@@ -1,65 +1,60 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- UNNEST  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- UNNEST  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (TestYelp.YelpCheckinB.YelpCheckinB)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (TestYelp.YelpCheckinB.YelpCheckinB)  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
+                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                    -- STABLE_SORT [$$84(ASC), $$85(ASC)]  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STABLE_SORT [$$80(ASC), $$81(ASC)]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- BTREE_SEARCH (TestYelp.YelpCheckinB.IdxYelpCheckinBDates)  |PARTITIONED|
-                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                    -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- STREAM_SELECT  |PARTITIONED|
+                                            -- BTREE_SEARCH (TestYelp.YelpCheckinB.IdxYelpCheckinBDates)  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- SUBPLAN  |PARTITIONED|
+                                                              {
+                                                                -- AGGREGATE  |LOCAL|
+                                                                  -- STREAM_SELECT  |LOCAL|
+                                                                    -- UNNEST  |LOCAL|
+                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                              }
+                                                        -- ASSIGN  |PARTITIONED|
                                                           -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- SUBPLAN  |PARTITIONED|
-                                                                    {
-                                                                      -- AGGREGATE  |LOCAL|
-                                                                        -- STREAM_SELECT  |LOCAL|
-                                                                          -- UNNEST  |LOCAL|
-                                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                    }
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH (TestYelp.YelpCheckinA.YelpCheckinA)  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                   -- STREAM_PROJECT  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- BTREE_SEARCH (TestYelp.YelpCheckinA.YelpCheckinA)  |PARTITIONED|
+                                                                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- STABLE_SORT [$$82(ASC), $$72(ASC)]  |PARTITIONED|
                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                                                              -- STREAM_PROJECT  |PARTITIONED|
                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- STABLE_SORT [$$78(ASC), $$68(ASC)]  |PARTITIONED|
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                  -- BTREE_SEARCH (TestYelp.YelpCheckinA.IdxYelpCheckinADates)  |PARTITIONED|
+                                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                          -- BTREE_SEARCH (TestYelp.YelpCheckinA.IdxYelpCheckinADates)  |PARTITIONED|
-                                                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                    -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                          -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query5.plan
index e06e50b..d6a912e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query5.plan
@@ -1,38 +1,35 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- UNNEST  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- UNNEST  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                      -- STABLE_SORT [$$75(ASC), $$67(ASC)]  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STABLE_SORT [$$71(ASC), $$63(ASC)]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                              -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-2/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-2/query1.plan
index 641e620..5832cff 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-2/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-2/query1.plan
@@ -1,39 +1,36 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- UNNEST  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- UNNEST  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                      -- STABLE_SORT [$$66(ASC), $$57(ASC)]  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STABLE_SORT [$$63(ASC), $$54(ASC)]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                              -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-2/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-2/query2.plan
index 109cc05..d276431 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-2/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-2/query2.plan
@@ -1,38 +1,35 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- UNNEST  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- UNNEST  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                      -- STABLE_SORT [$$77(ASC), $$68(ASC)]  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STABLE_SORT [$$73(ASC), $$64(ASC)]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                              -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-2/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-2/query3.plan
index d895e99..ba0454c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-2/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-2/query3.plan
@@ -1,35 +1,32 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- UNNEST  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- NESTED_LOOP  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- UNNEST  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- NESTED_LOOP  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_SELECT  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |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/array-index/join-quantified-queries/use-case-3/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-3/query1.plan
index 09bcc02..f12c1c5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-3/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-3/query1.plan
@@ -1,40 +1,37 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- ASSIGN  |LOCAL|
-                                  -- UNNEST  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- ASSIGN  |LOCAL|
+                                -- UNNEST  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                      -- STABLE_SORT [$$75(ASC), $$66(ASC)]  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STABLE_SORT [$$71(ASC), $$62(ASC)]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                              -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-3/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-3/query2.plan
index 7217e9f..19dc505 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-3/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-3/query2.plan
@@ -1,39 +1,36 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- ASSIGN  |LOCAL|
-                                -- UNNEST  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- ASSIGN  |LOCAL|
+                              -- UNNEST  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                      -- STABLE_SORT [$$77(ASC), $$68(ASC)]  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STABLE_SORT [$$73(ASC), $$64(ASC)]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                              -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query1.plan
index 22f9aa6..36c2e82 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query1.plan
@@ -1,47 +1,44 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- SUBPLAN  |LOCAL|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- STREAM_SELECT  |LOCAL|
-                                              -- UNNEST  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- ASSIGN  |LOCAL|
-                                    -- UNNEST  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- SUBPLAN  |LOCAL|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- UNNEST  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ASSIGN  |LOCAL|
+                                  -- UNNEST  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                      -- STABLE_SORT [$$89(ASC), $$78(ASC)]  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STABLE_SORT [$$84(ASC), $$73(ASC)]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                              -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query2.plan
index df73479..453abb1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query2.plan
@@ -1,47 +1,44 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- SUBPLAN  |LOCAL|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- STREAM_SELECT  |LOCAL|
-                                              -- UNNEST  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- ASSIGN  |LOCAL|
-                                    -- UNNEST  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- SUBPLAN  |LOCAL|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- UNNEST  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ASSIGN  |LOCAL|
+                                  -- UNNEST  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                      -- STABLE_SORT [$$93(ASC), $$81(ASC)]  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STABLE_SORT [$$88(ASC), $$76(ASC)]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                              -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query3.plan
index 6c55934..aa9e6b0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query3.plan
@@ -1,45 +1,42 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- SUBPLAN  |LOCAL|
-                                      {
-                                        -- AGGREGATE  |LOCAL|
-                                          -- UNNEST  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                      }
-                                -- ASSIGN  |LOCAL|
-                                  -- UNNEST  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- SUBPLAN  |LOCAL|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- UNNEST  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ASSIGN  |LOCAL|
+                                -- UNNEST  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                      -- STABLE_SORT [$$93(ASC), $$82(ASC)]  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STABLE_SORT [$$88(ASC), $$77(ASC)]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                              -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query4.plan
index 552ba9b..696a0f0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query4.plan
@@ -1,77 +1,72 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- SUBPLAN  |LOCAL|
-                                      {
-                                        -- AGGREGATE  |LOCAL|
-                                          -- UNNEST  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                      }
-                                -- ASSIGN  |LOCAL|
-                                  -- UNNEST  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- SUBPLAN  |LOCAL|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- UNNEST  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ASSIGN  |LOCAL|
+                                -- UNNEST  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (TestYelp.YelpCheckinB.YelpCheckinB)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (TestYelp.YelpCheckinB.YelpCheckinB)  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
+                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                    -- STABLE_SORT [$$142(ASC), $$143(ASC)]  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STABLE_SORT [$$134(ASC), $$135(ASC)]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- BTREE_SEARCH (TestYelp.YelpCheckinB.IdxYelpCheckinDatesB)  |PARTITIONED|
-                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                    -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- STREAM_SELECT  |PARTITIONED|
+                                            -- BTREE_SEARCH (TestYelp.YelpCheckinB.IdxYelpCheckinDatesB)  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- SUBPLAN  |PARTITIONED|
+                                                              {
+                                                                -- AGGREGATE  |LOCAL|
+                                                                  -- SUBPLAN  |LOCAL|
+                                                                          {
+                                                                            -- AGGREGATE  |LOCAL|
+                                                                              -- UNNEST  |LOCAL|
+                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                          }
+                                                                    -- ASSIGN  |LOCAL|
+                                                                      -- UNNEST  |LOCAL|
+                                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                              }
+                                                        -- ASSIGN  |PARTITIONED|
                                                           -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- SUBPLAN  |PARTITIONED|
-                                                                    {
-                                                                      -- AGGREGATE  |LOCAL|
-                                                                        -- SUBPLAN  |LOCAL|
-                                                                                {
-                                                                                  -- AGGREGATE  |LOCAL|
-                                                                                    -- UNNEST  |LOCAL|
-                                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                }
-                                                                          -- ASSIGN  |LOCAL|
-                                                                            -- UNNEST  |LOCAL|
-                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                    }
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH (TestYelp.YelpCheckinA.YelpCheckinA)  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                   -- STREAM_PROJECT  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- BTREE_SEARCH (TestYelp.YelpCheckinA.YelpCheckinA)  |PARTITIONED|
+                                                                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- STABLE_SORT [$$140(ASC), $$124(ASC)]  |PARTITIONED|
                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                                                              -- STREAM_PROJECT  |PARTITIONED|
                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- STABLE_SORT [$$132(ASC), $$116(ASC)]  |PARTITIONED|
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                  -- BTREE_SEARCH (TestYelp.YelpCheckinA.IdxYelpCheckinDatesA)  |PARTITIONED|
+                                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                          -- BTREE_SEARCH (TestYelp.YelpCheckinA.IdxYelpCheckinDatesA)  |PARTITIONED|
-                                                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                    -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                          -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query1.plan
index fb3c603..1f0fd3d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query1.plan
@@ -1,32 +1,27 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
           -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- STABLE_SORT [$$52(ASC), $$53(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$49(ASC), $$50(ASC)]  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                  -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- RUNNING_AGGREGATE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query2.plan
index 70118a1..2a54768 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query2.plan
@@ -1,32 +1,27 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+                            -- STABLE_SORT [$$52(ASC), $$53(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$49(ASC), $$50(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        -- RUNNING_AGGREGATE  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query3.plan
index bb23941..15fb51a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query3.plan
@@ -1,34 +1,30 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STABLE_SORT [$$75(ASC), $$76(ASC)]  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                    -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$80(ASC), $$81(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query4.plan
index f602149..91ae9ed 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query4.plan
@@ -1,37 +1,32 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- WINDOW_STREAM  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$70(ASC), $$71(DESC)]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$70]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- WINDOW_STREAM  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$75(ASC), $$76(DESC)]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$75]  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- LEFT_OUTER_UNNEST  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
-                                -- LEFT_OUTER_UNNEST  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                          -- STABLE_SORT [$$84(ASC), $$75(ASC)]  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STABLE_SORT [$$79(ASC), $$70(ASC)]  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query1.plan
index 2b052dc..9627fd2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query1.plan
@@ -1,32 +1,27 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- STABLE_SORT [$$54(ASC), $$55(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$51(ASC), $$52(ASC)]  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                  -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- RUNNING_AGGREGATE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query2.plan
index 1cad651..be41b71 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query2.plan
@@ -1,32 +1,27 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+                            -- STABLE_SORT [$$54(ASC), $$55(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$51(ASC), $$52(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        -- RUNNING_AGGREGATE  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query3.plan
index 36d1e39..57f75e8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query3.plan
@@ -1,34 +1,30 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STABLE_SORT [$$77(ASC), $$78(ASC)]  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                    -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$82(ASC), $$83(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query4.plan
index 44fd353..9762289 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query4.plan
@@ -1,37 +1,32 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- WINDOW_STREAM  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$71(ASC), $$72(DESC)]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$71]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- WINDOW_STREAM  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$76(ASC), $$77(DESC)]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$76]  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- LEFT_OUTER_UNNEST  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
-                                -- LEFT_OUTER_UNNEST  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                          -- STABLE_SORT [$$86(ASC), $$76(ASC)]  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STABLE_SORT [$$81(ASC), $$71(ASC)]  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query1.plan
index bf67e63..1446a4f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query1.plan
@@ -1,32 +1,27 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
           -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- STABLE_SORT [$$54(ASC), $$55(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$51(ASC), $$52(ASC)]  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                  -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- RUNNING_AGGREGATE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query2.plan
index 443990e..b5eeda1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query2.plan
@@ -1,32 +1,27 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- UNNEST  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
               -- STREAM_PROJECT  |PARTITIONED|
-                -- UNNEST  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- STABLE_SORT [$$54(ASC), $$55(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$51(ASC), $$52(ASC)]  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                  -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- RUNNING_AGGREGATE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query3.plan
index 1cad651..be41b71 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query3.plan
@@ -1,32 +1,27 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+                            -- STABLE_SORT [$$54(ASC), $$55(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$51(ASC), $$52(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        -- RUNNING_AGGREGATE  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query4.plan
index 36d1e39..57f75e8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query4.plan
@@ -1,34 +1,30 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STABLE_SORT [$$77(ASC), $$78(ASC)]  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                    -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$82(ASC), $$83(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query5.plan
index 0497e66..98dd5f0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query5.plan
@@ -1,37 +1,32 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- WINDOW_STREAM  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$72(ASC), $$73(DESC)]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$72]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- WINDOW_STREAM  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$77(ASC), $$78(DESC)]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$77]  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- LEFT_OUTER_UNNEST  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
-                                -- LEFT_OUTER_UNNEST  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                          -- STABLE_SORT [$$86(ASC), $$77(ASC)]  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STABLE_SORT [$$81(ASC), $$72(ASC)]  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query1.plan
index e09340f..81495ee 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query1.plan
@@ -1,36 +1,29 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- UNNEST  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- STREAM_SELECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- UNNEST  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                  -- STABLE_SORT [$$65(ASC), $$66(ASC)]  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STABLE_SORT [$$61(ASC), $$62(ASC)]  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                          -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query2.plan
index 1569695..01f899a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query2.plan
@@ -1,37 +1,30 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
           -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
               -- UNNEST  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- UNNEST  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                    -- STABLE_SORT [$$68(ASC), $$69(ASC)]  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STABLE_SORT [$$64(ASC), $$65(ASC)]  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                            -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query3.plan
index c4d7c08..ca0f198 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query3.plan
@@ -1,36 +1,29 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- UNNEST  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- UNNEST  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                -- STABLE_SORT [$$65(ASC), $$66(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$61(ASC), $$62(ASC)]  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            -- RUNNING_AGGREGATE  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                        -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- STREAM_SELECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query4.plan
index 34a1b94..71f0b32 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query4.plan
@@ -1,38 +1,32 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- UNNEST  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STABLE_SORT [$$87(ASC), $$88(ASC)]  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                            -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$93(ASC), $$94(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                  -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query5.plan
index aac007e..d86fa28 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query5.plan
@@ -1,41 +1,34 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- WINDOW_STREAM  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$81(ASC), $$82(DESC)]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$81]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- LEFT_OUTER_UNNEST  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- WINDOW_STREAM  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$87(ASC), $$88(DESC)]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$87]  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- LEFT_OUTER_UNNEST  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- LEFT_OUTER_UNNEST  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- LEFT_OUTER_UNNEST  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STABLE_SORT [$$97(ASC), $$87(ASC)]  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- STABLE_SORT [$$91(ASC), $$81(ASC)]  |PARTITIONED|
+                                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        -- ASSIGN  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers)  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-indexes/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-indexes/query1.plan
index 0d37f51..fd56b21 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-indexes/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-indexes/query1.plan
@@ -1,43 +1,41 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- ASSIGN  |LOCAL|
-                              -- UNNEST  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- ASSIGN  |LOCAL|
+                          -- UNNEST  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (test.KSI.KSI)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (test.KSI.KSI)  |PARTITIONED|
+                      -- INTERSECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- INTERSECT  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                              -- STABLE_SORT [$$40(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$40(ASC)]  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                      -- BTREE_SEARCH (test.KSI.KS1_array_index1)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (test.KSI.KS1_array_index1)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                              -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                      -- BTREE_SEARCH (test.KSI.KS1_array_index2)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (test.KSI.KS1_array_index2)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-indexes/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-indexes/query2.plan
index b793939..e381267 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-indexes/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-indexes/query2.plan
@@ -1,54 +1,52 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- ASSIGN  |LOCAL|
-                              -- UNNEST  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- ASSIGN  |LOCAL|
+                          -- UNNEST  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (test.KSI.KSI)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (test.KSI.KSI)  |PARTITIONED|
+                      -- INTERSECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- INTERSECT  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                              -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                      -- BTREE_SEARCH (test.KSI.KS1_array_index1)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (test.KSI.KS1_array_index1)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                              -- STABLE_SORT [$$47(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$47(ASC)]  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                      -- BTREE_SEARCH (test.KSI.KS1_array_index2)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (test.KSI.KS1_array_index2)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                              -- STABLE_SORT [$$51(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$51(ASC)]  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                      -- BTREE_SEARCH (test.KSI.KS1_array_index3)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (test.KSI.KS1_array_index3)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query1.plan
index 62b49d7..305f6e9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query1.plan
@@ -1,31 +1,29 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- UNNEST  |LOCAL|
                           -- STREAM_SELECT  |LOCAL|
                             -- UNNEST  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- UNNEST  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                          -- STABLE_SORT [$$49(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query10.plan
index 50d2d2d..2fc7fad 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query10.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query10.plan
@@ -1,36 +1,34 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- UNNEST  |LOCAL|
-                              -- SUBPLAN  |LOCAL|
-                                      {
-                                        -- AGGREGATE  |LOCAL|
-                                          -- STREAM_SELECT  |LOCAL|
-                                            -- UNNEST  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                      }
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- UNNEST  |LOCAL|
+                          -- SUBPLAN  |LOCAL|
+                                  {
+                                    -- AGGREGATE  |LOCAL|
+                                      -- STREAM_SELECT  |LOCAL|
+                                        -- UNNEST  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                  }
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                          -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query11.plan
index f14e55a..8dacd0f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query11.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query11.plan
@@ -1,43 +1,41 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- UNNEST  |LOCAL|
-                              -- SUBPLAN  |LOCAL|
-                                      {
-                                        -- AGGREGATE  |LOCAL|
-                                          -- STREAM_SELECT  |LOCAL|
-                                            -- UNNEST  |LOCAL|
-                                              -- SUBPLAN  |LOCAL|
-                                                      {
-                                                        -- AGGREGATE  |LOCAL|
-                                                          -- STREAM_SELECT  |LOCAL|
-                                                            -- UNNEST  |LOCAL|
-                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                      }
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                      }
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- UNNEST  |LOCAL|
+                          -- SUBPLAN  |LOCAL|
+                                  {
+                                    -- AGGREGATE  |LOCAL|
+                                      -- STREAM_SELECT  |LOCAL|
+                                        -- UNNEST  |LOCAL|
+                                          -- SUBPLAN  |LOCAL|
+                                                  {
+                                                    -- AGGREGATE  |LOCAL|
+                                                      -- STREAM_SELECT  |LOCAL|
+                                                        -- UNNEST  |LOCAL|
+                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                  }
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                  }
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                          -- STABLE_SORT [$$41(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$41(ASC)]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query2.plan
index 62b49d7..305f6e9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query2.plan
@@ -1,31 +1,29 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- UNNEST  |LOCAL|
                           -- STREAM_SELECT  |LOCAL|
                             -- UNNEST  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- UNNEST  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                          -- STABLE_SORT [$$49(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query3.plan
index 24b6a5a..2f27a36 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query3.plan
@@ -1,33 +1,31 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- ASSIGN  |LOCAL|
-                              -- UNNEST  |LOCAL|
-                                -- STREAM_SELECT  |LOCAL|
-                                  -- ASSIGN  |LOCAL|
-                                    -- UNNEST  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- ASSIGN  |LOCAL|
+                          -- UNNEST  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- ASSIGN  |LOCAL|
+                                -- UNNEST  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                          -- STABLE_SORT [$$53(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query4.plan
index 24b6a5a..2f27a36 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query4.plan
@@ -1,33 +1,31 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- ASSIGN  |LOCAL|
-                              -- UNNEST  |LOCAL|
-                                -- STREAM_SELECT  |LOCAL|
-                                  -- ASSIGN  |LOCAL|
-                                    -- UNNEST  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- ASSIGN  |LOCAL|
+                          -- UNNEST  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- ASSIGN  |LOCAL|
+                                -- UNNEST  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                          -- STABLE_SORT [$$53(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query5.plan
index 7992dcf..68e9493 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query5.plan
@@ -1,44 +1,42 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- SUBPLAN  |LOCAL|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- ASSIGN  |LOCAL|
+                                        -- UNNEST  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- ASSIGN  |LOCAL|
+                                              -- UNNEST  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
                           -- STREAM_SELECT  |LOCAL|
-                            -- SUBPLAN  |LOCAL|
-                                    {
-                                      -- AGGREGATE  |LOCAL|
-                                        -- STREAM_SELECT  |LOCAL|
-                                          -- ASSIGN  |LOCAL|
-                                            -- UNNEST  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- ASSIGN  |LOCAL|
-                                                  -- UNNEST  |LOCAL|
-                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                    }
-                              -- STREAM_SELECT  |LOCAL|
+                            -- ASSIGN  |LOCAL|
+                              -- UNNEST  |LOCAL|
                                 -- ASSIGN  |LOCAL|
                                   -- UNNEST  |LOCAL|
-                                    -- ASSIGN  |LOCAL|
-                                      -- UNNEST  |LOCAL|
-                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                          -- STABLE_SORT [$$86(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$81(ASC)]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query6.plan
index 7992dcf..68e9493 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query6.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query6.plan
@@ -1,44 +1,42 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- SUBPLAN  |LOCAL|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- ASSIGN  |LOCAL|
+                                        -- UNNEST  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- ASSIGN  |LOCAL|
+                                              -- UNNEST  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
                           -- STREAM_SELECT  |LOCAL|
-                            -- SUBPLAN  |LOCAL|
-                                    {
-                                      -- AGGREGATE  |LOCAL|
-                                        -- STREAM_SELECT  |LOCAL|
-                                          -- ASSIGN  |LOCAL|
-                                            -- UNNEST  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- ASSIGN  |LOCAL|
-                                                  -- UNNEST  |LOCAL|
-                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                    }
-                              -- STREAM_SELECT  |LOCAL|
+                            -- ASSIGN  |LOCAL|
+                              -- UNNEST  |LOCAL|
                                 -- ASSIGN  |LOCAL|
                                   -- UNNEST  |LOCAL|
-                                    -- ASSIGN  |LOCAL|
-                                      -- UNNEST  |LOCAL|
-                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                          -- STABLE_SORT [$$86(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$81(ASC)]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query7.plan
index 7999e77..4f6e5e9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query7.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query7.plan
@@ -1,31 +1,29 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- ASSIGN  |LOCAL|
+                        -- UNNEST  |LOCAL|
                           -- ASSIGN  |LOCAL|
                             -- UNNEST  |LOCAL|
-                              -- ASSIGN  |LOCAL|
-                                -- UNNEST  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                          -- STABLE_SORT [$$55(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query8.plan
index cc99bbf..95e3750 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query8.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query8.plan
@@ -1,40 +1,38 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- SUBPLAN  |LOCAL|
-                                  {
-                                    -- AGGREGATE  |LOCAL|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- SUBPLAN  |LOCAL|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- ASSIGN  |LOCAL|
+                                    -- UNNEST  |LOCAL|
                                       -- ASSIGN  |LOCAL|
                                         -- UNNEST  |LOCAL|
-                                          -- ASSIGN  |LOCAL|
-                                            -- UNNEST  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                  }
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ASSIGN  |LOCAL|
+                          -- UNNEST  |LOCAL|
                             -- ASSIGN  |LOCAL|
                               -- UNNEST  |LOCAL|
-                                -- ASSIGN  |LOCAL|
-                                  -- UNNEST  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                          -- STABLE_SORT [$$90(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$85(ASC)]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query9.plan
index c9574d6..1c9bce9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query9.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query9.plan
@@ -1,40 +1,38 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- SUBPLAN  |LOCAL|
-                                  {
-                                    -- AGGREGATE  |LOCAL|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- SUBPLAN  |LOCAL|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- ASSIGN  |LOCAL|
+                                    -- UNNEST  |LOCAL|
                                       -- ASSIGN  |LOCAL|
                                         -- UNNEST  |LOCAL|
-                                          -- ASSIGN  |LOCAL|
-                                            -- UNNEST  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                  }
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ASSIGN  |LOCAL|
+                          -- UNNEST  |LOCAL|
                             -- ASSIGN  |LOCAL|
                               -- UNNEST  |LOCAL|
-                                -- ASSIGN  |LOCAL|
-                                  -- UNNEST  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                          -- STABLE_SORT [$$93(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$88(ASC)]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query1.plan
index 2c5b278..cec77d1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query1.plan
@@ -1,30 +1,27 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- UNNEST  |LOCAL|
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- UNNEST  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query2.plan
index 761ed03..a0b5dd6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query2.plan
@@ -1,32 +1,30 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- UNNEST  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- UNNEST  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$55(ASC)]  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
+                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query3.plan
index f2819ea..247ddd3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query3.plan
@@ -1,33 +1,31 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- UNNEST  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- UNNEST  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                  -- STABLE_SORT [$$63(ASC)]  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                          -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query4.plan
index 1ed6cd6..767e1cc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query4.plan
@@ -1,31 +1,29 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- UNNEST  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- UNNEST  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$57(ASC)]  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
+                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query5.plan
index 9120584..219bd40 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query5.plan
@@ -1,24 +1,22 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- UNNEST  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- UNNEST  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query1.plan
index e29b93c..2945638 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query1.plan
@@ -1,30 +1,27 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- UNNEST  |LOCAL|
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- UNNEST  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query2.plan
index 9843256..6f63beb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query2.plan
@@ -1,32 +1,30 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- UNNEST  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- UNNEST  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$57(ASC)]  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
+                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query3.plan
index 10ad036..63f05f1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query3.plan
@@ -1,31 +1,29 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- UNNEST  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- UNNEST  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                -- STABLE_SORT [$$62(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$59(ASC)]  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
+                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query4.plan
index 2d561f4..48dfd6c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query4.plan
@@ -1,22 +1,20 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- UNNEST  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- UNNEST  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query5.plan
index 2d561f4..48dfd6c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query5.plan
@@ -1,22 +1,20 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- UNNEST  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- UNNEST  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query1.plan
index 1d92106..a600cfb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query1.plan
@@ -1,31 +1,28 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- ASSIGN  |LOCAL|
-                              -- UNNEST  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- ASSIGN  |LOCAL|
+                          -- UNNEST  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query2.plan
index 45549a5..db9661a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query2.plan
@@ -1,31 +1,28 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- ASSIGN  |LOCAL|
-                              -- UNNEST  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$40(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- ASSIGN  |LOCAL|
+                          -- UNNEST  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query3.plan
index 2fb6ed5..387d9d4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query3.plan
@@ -1,30 +1,27 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- ASSIGN  |LOCAL|
-                            -- UNNEST  |LOCAL|
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- ASSIGN  |LOCAL|
+                        -- UNNEST  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query4.plan
index 7e503bb..ff93394 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query4.plan
@@ -1,21 +1,18 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- ASSIGN  |LOCAL|
-                            -- UNNEST  |LOCAL|
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- ASSIGN  |LOCAL|
+                        -- UNNEST  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query1.plan
index 84dc8c7..82fc9d4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query1.plan
@@ -1,38 +1,35 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- SUBPLAN  |LOCAL|
-                                    {
-                                      -- AGGREGATE  |LOCAL|
-                                        -- STREAM_SELECT  |LOCAL|
-                                          -- UNNEST  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                    }
-                              -- ASSIGN  |LOCAL|
-                                -- UNNEST  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$41(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- SUBPLAN  |LOCAL|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- UNNEST  |LOCAL|
+                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ASSIGN  |LOCAL|
+                            -- UNNEST  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query2.plan
index a689c35..8b84234 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query2.plan
@@ -1,38 +1,35 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- SUBPLAN  |LOCAL|
-                                    {
-                                      -- AGGREGATE  |LOCAL|
-                                        -- STREAM_SELECT  |LOCAL|
-                                          -- UNNEST  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                    }
-                              -- ASSIGN  |LOCAL|
-                                -- UNNEST  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$49(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- SUBPLAN  |LOCAL|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- UNNEST  |LOCAL|
+                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ASSIGN  |LOCAL|
+                            -- UNNEST  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query3.plan
index 8de78d4..387dacd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query3.plan
@@ -1,37 +1,34 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- SUBPLAN  |LOCAL|
-                                  {
-                                    -- AGGREGATE  |LOCAL|
-                                      -- STREAM_SELECT  |LOCAL|
-                                        -- UNNEST  |LOCAL|
-                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                  }
-                            -- ASSIGN  |LOCAL|
-                              -- UNNEST  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$51(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- SUBPLAN  |LOCAL|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- UNNEST  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ASSIGN  |LOCAL|
+                          -- UNNEST  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query4.plan
index 10b415f..7bba772 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query4.plan
@@ -1,37 +1,34 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- SUBPLAN  |LOCAL|
-                                    {
-                                      -- AGGREGATE  |LOCAL|
-                                        -- UNNEST  |LOCAL|
-                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                    }
-                              -- ASSIGN  |LOCAL|
-                                -- UNNEST  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- SUBPLAN  |LOCAL|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- UNNEST  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ASSIGN  |LOCAL|
+                            -- UNNEST  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$53(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query5.plan
index f1c1609..2342669 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query5.plan
@@ -1,36 +1,33 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- SUBPLAN  |LOCAL|
-                                  {
-                                    -- AGGREGATE  |LOCAL|
-                                      -- UNNEST  |LOCAL|
-                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                  }
-                            -- ASSIGN  |LOCAL|
-                              -- UNNEST  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- SUBPLAN  |LOCAL|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- UNNEST  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ASSIGN  |LOCAL|
+                          -- UNNEST  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$55(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query6.plan
index 3f101b7..9d9cc3f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query6.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query6.plan
@@ -1,29 +1,26 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- SUBPLAN  |LOCAL|
-                                    {
-                                      -- AGGREGATE  |LOCAL|
-                                        -- STREAM_SELECT  |LOCAL|
-                                          -- UNNEST  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                    }
-                              -- ASSIGN  |LOCAL|
-                                -- UNNEST  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- SUBPLAN  |LOCAL|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- UNNEST  |LOCAL|
+                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ASSIGN  |LOCAL|
+                            -- UNNEST  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query7.plan
index 0f3a84c..96808dc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query7.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query7.plan
@@ -1,39 +1,36 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- SUBPLAN  |LOCAL|
-                                    {
-                                      -- AGGREGATE  |LOCAL|
-                                        -- STREAM_SELECT  |LOCAL|
-                                          -- UNNEST  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                    }
-                              -- ASSIGN  |LOCAL|
-                                -- UNNEST  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- SUBPLAN  |LOCAL|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- UNNEST  |LOCAL|
+                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ASSIGN  |LOCAL|
+                            -- UNNEST  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$65(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-1/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-1/query1.plan
index 85a80b7..180df58 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-1/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-1/query1.plan
@@ -1,23 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-1/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-1/query2.plan
index a65a2c1..d18544a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-1/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-1/query2.plan
@@ -1,26 +1,23 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$57(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-2/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-2/query1.plan
index 3f9647b..18867f9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-2/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-2/query1.plan
@@ -1,23 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$37(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-2/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-2/query2.plan
index 9af45a6..aec2f23 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-2/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-2/query2.plan
@@ -1,26 +1,23 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$56(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$59(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query1.plan
index 3f9647b..18867f9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query1.plan
@@ -1,23 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$37(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query2.plan
index bb72552..ebbf2ae 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query2.plan
@@ -1,28 +1,24 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- UNNEST  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- UNNEST  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STABLE_SORT [$$57(ASC)]  |PARTITIONED|
+                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query3.plan
index 0202127..f7d515f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query3.plan
@@ -1,23 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$41(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-4/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-4/query1.plan
index 4cf8dfe..2d66d75 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-4/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-4/query1.plan
@@ -1,27 +1,22 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- UNNEST  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- UNNEST  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-4/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-4/query2.plan
index 4cb1efa..1363a50 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-4/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-4/query2.plan
@@ -1,30 +1,25 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- UNNEST  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$66(ASC)]  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$70(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.plan
index 0202127..f7d515f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.plan
@@ -1,23 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$41(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.plan
index 9177af3..380c782 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.plan
@@ -1,26 +1,23 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$63(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-pk/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-pk/query1.plan
index 883fa06..6835463 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-pk/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-pk/query1.plan
@@ -1,24 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- STABLE_SORT [$$36(ASC), $$37(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+                                -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-pk/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-pk/query2.plan
index e4d4667..c565154 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-pk/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-pk/query2.plan
@@ -1,26 +1,23 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$55(ASC), $$56(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$58(ASC), $$59(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-sk/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-sk/query1.plan
index 2ad5dca..560f87d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-sk/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-sk/query1.plan
@@ -1,25 +1,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
           -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                          -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDatesTimes)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDatesTimes)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-sk/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-sk/query2.plan
index afcd3a0..9c70594 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-sk/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-sk/query2.plan
@@ -1,28 +1,24 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- UNNEST  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- UNNEST  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
+                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDatesTimes)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDatesTimes)  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-sk/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-sk/query3.plan
index 4af61e1..a2530d6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-sk/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-sk/query3.plan
@@ -1,19 +1,15 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- UNNEST  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- UNNEST  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-filter-fields/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-filter-fields/query1.plan
index 3a14007..9e3ae2b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-filter-fields/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-filter-fields/query1.plan
@@ -1,25 +1,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
           -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                          -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-filter-fields/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-filter-fields/query2.plan
index a65a2c1..d18544a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-filter-fields/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-filter-fields/query2.plan
@@ -1,26 +1,23 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$57(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query1.plan
index 0d24b2d..a585a44 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query1.plan
@@ -1,22 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- UNNEST  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset)  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset)  |PARTITIONED|
+                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                        -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
+                                -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex1)  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex1)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query2.plan
index 3585861..4c51096 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query2.plan
@@ -1,22 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- UNNEST  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset)  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset)  |PARTITIONED|
+                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                        -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
+                                -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex2)  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex2)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query3.plan
index 679b2a0..d576fc3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query3.plan
@@ -1,25 +1,22 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- UNNEST  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex3)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$49(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex3)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query4.plan
index 3f26cee..f22a65a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query4.plan
@@ -1,25 +1,22 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- UNNEST  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex4)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$51(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex4)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query5.plan
index f1e0798..85d939a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query5.plan
@@ -1,22 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- UNNEST  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset)  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset)  |PARTITIONED|
+                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                        -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
+                                -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex1c)  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex1c)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query6.plan
index a31846b..1b44a26 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query6.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query6.plan
@@ -1,22 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- UNNEST  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset)  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset)  |PARTITIONED|
+                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                        -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
+                                -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex2c)  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex2c)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query7.plan
index 4ee8831..9ecb9c8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query7.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query7.plan
@@ -1,25 +1,22 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- UNNEST  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex3c)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$49(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex3c)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query8.plan
index c512a31..b6c6d86 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query8.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query8.plan
@@ -1,25 +1,22 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- UNNEST  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex4c)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$51(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex4c)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/multiple-indexes/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/multiple-indexes/query1.plan
index 4c76ea0..6b57fed 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/multiple-indexes/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/multiple-indexes/query1.plan
@@ -1,35 +1,33 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- UNNEST  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.KSI.KSI)  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH (test.KSI.KSI)  |PARTITIONED|
+                    -- INTERSECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- INTERSECT  |PARTITIONED|
+                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.KSI.KS1_array_index1)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.KSI.KS1_array_index1)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.KSI.KS1_array_index2)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.KSI.KS1_array_index2)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/multiple-indexes/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/multiple-indexes/query2.plan
index 7a2f7ce..c8fbf32 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/multiple-indexes/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/multiple-indexes/query2.plan
@@ -1,46 +1,44 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- UNNEST  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.KSI.KSI)  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH (test.KSI.KSI)  |PARTITIONED|
+                    -- INTERSECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- INTERSECT  |PARTITIONED|
+                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.KSI.KS1_array_index1)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.KSI.KS1_array_index1)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.KSI.KS1_array_index2)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.KSI.KS1_array_index2)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.KSI.KS1_array_index3)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.KSI.KS1_array_index3)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-1/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-1/query1.plan
index 85a80b7..180df58 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-1/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-1/query1.plan
@@ -1,23 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-1/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-1/query2.plan
index a65a2c1..d18544a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-1/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-1/query2.plan
@@ -1,26 +1,23 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$57(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-2/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-2/query1.plan
index 3f9647b..18867f9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-2/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-2/query1.plan
@@ -1,23 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$37(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-2/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-2/query2.plan
index 9af45a6..aec2f23 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-2/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-2/query2.plan
@@ -1,26 +1,23 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$56(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$59(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query1.plan
index 3f9647b..18867f9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query1.plan
@@ -1,23 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$37(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query2.plan
index bb72552..ebbf2ae 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query2.plan
@@ -1,28 +1,24 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- UNNEST  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- UNNEST  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                                -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STABLE_SORT [$$57(ASC)]  |PARTITIONED|
+                                        -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query3.plan
index 0202127..f7d515f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query3.plan
@@ -1,23 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$41(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-4/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-4/query1.plan
index 4cf8dfe..2d66d75 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-4/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-4/query1.plan
@@ -1,27 +1,22 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- UNNEST  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- UNNEST  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-4/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-4/query2.plan
index 4cb1efa..1363a50 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-4/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-4/query2.plan
@@ -1,30 +1,25 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- UNNEST  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$66(ASC)]  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$70(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-3-level-record-path/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-3-level-record-path/query1.plan
index 0202127..f7d515f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-3-level-record-path/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-3-level-record-path/query1.plan
@@ -1,23 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$41(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-3-level-record-path/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-3-level-record-path/query2.plan
index 9177af3..380c782 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-3-level-record-path/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-3-level-record-path/query2.plan
@@ -1,26 +1,23 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$63(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-composite-sk/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-composite-sk/query1.plan
index 8ae90bf..560f87d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-composite-sk/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-composite-sk/query1.plan
@@ -1,25 +1,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
           -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin)  |PARTITIONED|
+                          -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                  -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDatesTimes)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDatesTimes)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/hints-indexnl-params/hints-indexnl-params-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/hints-indexnl-params/hints-indexnl-params-3.plan
index 9627dce..4e9bc29 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/hints-indexnl-params/hints-indexnl-params-3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/hints-indexnl-params/hints-indexnl-params-3.plan
@@ -1,25 +1,23 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$37(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$37(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$39(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk2.tenk2)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.tenk2.idx_tenk2_1k_2k)  |PARTITIONED|
-                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk2.tenk2)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.tenk2.idx_tenk2_1k_2k)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (test.tenk1.tenk1)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (test.tenk1.tenk1)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/hints-indexnl-params/hints-indexnl-params-4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/hints-indexnl-params/hints-indexnl-params-4.plan
index 29d7941..acfe0c3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/hints-indexnl-params/hints-indexnl-params-4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/hints-indexnl-params/hints-indexnl-params-4.plan
@@ -6,19 +6,17 @@
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- HYBRID_HASH_JOIN [$$36][$$37]  |PARTITIONED|
               -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (test.tenk1.tenk1)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.tenk2)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.tenk1.tenk1)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.tenk2)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/hints-indexnl-params/hints-indexnl-params-5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/hints-indexnl-params/hints-indexnl-params-5.plan
index 29d7941..acfe0c3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/hints-indexnl-params/hints-indexnl-params-5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/hints-indexnl-params/hints-indexnl-params-5.plan
@@ -6,19 +6,17 @@
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- HYBRID_HASH_JOIN [$$36][$$37]  |PARTITIONED|
               -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (test.tenk1.tenk1)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.tenk2)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.tenk1.tenk1)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.tenk2)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/hints-indexnl-params/hints-indexnl-params-6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/hints-indexnl-params/hints-indexnl-params-6.plan
index 29d7941..acfe0c3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/hints-indexnl-params/hints-indexnl-params-6.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/hints-indexnl-params/hints-indexnl-params-6.plan
@@ -6,19 +6,17 @@
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- HYBRID_HASH_JOIN [$$36][$$37]  |PARTITIONED|
               -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (test.tenk1.tenk1)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.tenk2)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.tenk1.tenk1)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.tenk2)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/hints-indexnl-params/hints-indexnl-params-7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/hints-indexnl-params/hints-indexnl-params-7.plan
index 29d7941..acfe0c3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/hints-indexnl-params/hints-indexnl-params-7.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/hints-indexnl-params/hints-indexnl-params-7.plan
@@ -6,19 +6,17 @@
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- HYBRID_HASH_JOIN [$$36][$$37]  |PARTITIONED|
               -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (test.tenk1.tenk1)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.tenk2)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.tenk1.tenk1)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.tenk2)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/hints-indexnl-params/hints-indexnl-params-8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/hints-indexnl-params/hints-indexnl-params-8.plan
index 758182a..8c3f5e9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/hints-indexnl-params/hints-indexnl-params-8.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/hints-indexnl-params/hints-indexnl-params-8.plan
@@ -1,25 +1,23 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$37(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$37(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$39(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk2.tenk2)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.tenk2.idx_tenk2_1k)  |PARTITIONED|
-                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk2.tenk2)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.tenk2.idx_tenk2_1k)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (test.tenk1.tenk1)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (test.tenk1.tenk1)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.plan
index aefe821..53c684d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.plan
@@ -1,113 +1,102 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$53(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$53(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$70]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- STREAM_SELECT  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$70(ASC), $$45(ASC)]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$70]  |PARTITIONED|
-                              -- UNION_ALL  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- STREAM_SELECT  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- SPLIT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- ASSIGN  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- STREAM_SELECT  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- SPLIT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$53(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$53(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$70]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- STREAM_SELECT  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$70(ASC), $$45(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$70]  |PARTITIONED|
+                            -- UNION_ALL  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$70]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- STREAM_SELECT  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$70(ASC), $$45(ASC)]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$70]  |PARTITIONED|
-                                        -- UNION_ALL  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- SPLIT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- SPLIT  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        -- ASSIGN  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- SPLIT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
                                                               -- ASSIGN  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- ASSIGN  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- SPLIT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$70]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$70(ASC), $$45(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$70]  |PARTITIONED|
+                                      -- UNION_ALL  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- SPLIT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- SPLIT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |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-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.plan
index c8f17db..78dd14c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.plan
@@ -1,34 +1,31 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$56(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$46]  |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]  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$58(ASC) ]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$48]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- STREAM_SELECT  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$48(ASC), $$49(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                            -- STABLE_SORT [$$63(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |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-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.plan
index ef866c9..65e8723 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.plan
@@ -1,77 +1,72 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$56(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$56(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$46]  |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]  |PARTITIONED|
-                              -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$58(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |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|
+                        -- STABLE_SORT [$$48(ASC), $$49(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- ASSIGN  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$46]  |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]  |PARTITIONED|
-                                        -- STREAM_SELECT  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$63(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |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|
+                                  -- STABLE_SORT [$$48(ASC), $$49(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                  -- STABLE_SORT [$$63(ASC)]  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ASSIGN  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |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-join/leftouterjoin-probe-pidx-with-join-btree-sidx_03-index-only.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_03-index-only.plan
index c6a637e..3ad3f60 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_03-index-only.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_03-index-only.plan
@@ -1,51 +1,44 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$67(ASC), $$42(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$67(ASC), $$42(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- WINDOW_STREAM  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$67(ASC), $$42(DESC)]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$67]  |PARTITIONED|
-                          -- UNION_ALL  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$67(ASC), $$42(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$67(ASC), $$42(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- WINDOW_STREAM  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$67(ASC), $$42(DESC)]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$67]  |PARTITIONED|
+                      -- UNION_ALL  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                        -- SPLIT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- SPLIT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- ASSIGN  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- SPLIT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
+                                            -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
                                                 -- ASSIGN  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                     -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                         -- ASSIGN  |PARTITIONED|
                                                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SPLIT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |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-join/leftouterjoin-probe-pidx-with-join-btree-sidx_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_03.plan
index dc89dd2..f6df175 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_03.plan
@@ -1,33 +1,29 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$39(ASC), $$40(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$39(ASC), $$40(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- WINDOW_STREAM  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$39(ASC), $$40(DESC)]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
-                          -- STREAM_SELECT  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$41(ASC), $$42(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$41(ASC), $$42(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- WINDOW_STREAM  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$41(ASC), $$42(DESC)]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$41]  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$56(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                       -- ASSIGN  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_04.plan
index cda45d0..528f21e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_04.plan
@@ -1,16 +1,15 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$30]  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.Orders)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$32]  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |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/btree-index-join/secondary-equi-join-multiindex.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan
index ac9158e..8f2091d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan
@@ -1,22 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- BTREE_SEARCH (test.FacebookMessages.FacebookMessages)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$49(ASC)]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.FacebookMessages.fbmIdxAutId)  |PARTITIONED|
-                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.FacebookMessages.FacebookMessages)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$51(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.FacebookMessages.fbmIdxAutId)  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.FacebookUsers.FacebookUsers)  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (test.FacebookUsers.FacebookUsers)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan
index fc29b18..4959113 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan
@@ -1,22 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |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|
-                -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
+                -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.CSX.title_index)  |PARTITIONED|
-                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                        -- BTREE_SEARCH (test.CSX.title_index)  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan
index 69bcc80..bd949cc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan
@@ -1,22 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |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|
-                -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.DBLP.title_index)  |PARTITIONED|
-                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                        -- BTREE_SEARCH (test.DBLP.title_index)  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_04.plan
index f730221..b75b258 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_04.plan
@@ -1,33 +1,31 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.testdst3.testdst3)  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- BTREE_SEARCH (test.testdst3.testdst3)  |PARTITIONED|
+                -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$49(ASC)]  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.testdst3.sec3_Idx)  |PARTITIONED|
-                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- HYBRID_HASH_JOIN [$$46][$$42]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$46]  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                        -- BTREE_SEARCH (test.testdst3.sec3_Idx)  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$49][$$45]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$49]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.testdst)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.testdst)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$45]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.testdst2)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.testdst2)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_05.plan
index f642053..66d63f1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_05.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_05.plan
@@ -1,18 +1,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- BTREE_SEARCH (test.testdst.testdst)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.testdst.sec_Idx)  |PARTITIONED|
-                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |UNPARTITIONED|
-                                -- UNNEST  |UNPARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.testdst.testdst)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.testdst.sec_Idx)  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |UNPARTITIONED|
+                              -- UNNEST  |UNPARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_06.plan
index f04f6a0..f2dbe69 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_06.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_06.plan
@@ -1,38 +1,34 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.testdst2.testdst2)  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- BTREE_SEARCH (test.testdst2.testdst2)  |PARTITIONED|
+                -- STABLE_SORT [$$85(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.testdst2.sec_Idx)  |PARTITIONED|
-                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- SORT_GROUP_BY[$$78]  |PARTITIONED|
-                                              {
-                                                -- AGGREGATE  |LOCAL|
-                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                              }
-                                        -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
-                                          -- SORT_GROUP_BY[$$71]  |PARTITIONED|
-                                                  {
-                                                    -- AGGREGATE  |LOCAL|
-                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                  }
+                        -- BTREE_SEARCH (test.testdst2.sec_Idx)  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$83]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$76]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (test.testdst)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.testdst)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/btree-index-composite-key-04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/btree-index-composite-key-04.plan
index 353b922..f55b79e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/btree-index-composite-key-04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/btree-index-composite-key-04.plan
@@ -1,29 +1,15 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$22(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- UNION_ALL  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.employee.employee)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- SPLIT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (test.employee.idx_employee_f_l_name)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$22(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- UNION_ALL  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.employee.employee)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- SPLIT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -31,3 +17,13 @@
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (test.employee.idx_employee_f_l_name)  |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-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-01.plan
index 17b0d76..c7e7795 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
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$61(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.ds1)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.ds1)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |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 f926a1c..cc2475f 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
@@ -1,21 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$61(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (test.ds2.ds2)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (test.ds2.ds2)  |PARTITIONED|
+                      -- STABLE_SORT [$$74(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$74(ASC)]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- BTREE_SEARCH (test.ds2.idx2)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.ds2.idx2)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |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 92b3b68..efabbca 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
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$61(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.ds3)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.ds3)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |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 88dc9b3..190d315 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
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$22(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.ds3)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.ds3)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |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-05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-05.plan
index 1c9845f..a0fb880 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-05.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-05.plan
@@ -1,27 +1,24 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$122(ASC), $$123(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$122(ASC), $$123(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$128(ASC), $$129(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$128(ASC), $$129(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (test.ds2.ds2)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (test.ds2.ds2)  |PARTITIONED|
+                        -- STABLE_SORT [$$145(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$139(ASC)]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (test.ds2.idx2)  |PARTITIONED|
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN (test.ds4)  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.ds2.idx2)  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.ds4)  |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-06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-06.plan
index 510ccca..af975ab 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-06.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-06.plan
@@ -1,24 +1,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$122(ASC), $$123(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$122(ASC), $$123(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- HYBRID_HASH_JOIN [$$122][$$123]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$128(ASC), $$129(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$128(ASC), $$129(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$128][$$129]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$128]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.ds4)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.ds4)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$123]  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$129]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.ds3)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.ds3)  |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/btree-index-selection/cast-default-null/cast-default-null-07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-07.plan
index 1577585..340321d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-07.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-07.plan
@@ -1,24 +1,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$122(ASC), $$123(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$122(ASC), $$123(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- HYBRID_HASH_JOIN [$$122][$$123]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$128(ASC), $$129(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$128(ASC), $$129(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$128][$$129]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$128]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.ds4)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.ds4)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$123]  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$129]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.ds1)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.ds1)  |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/btree-index-selection/cast-default-null/cast-default-null-08.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-08.plan
index 1577585..340321d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-08.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-08.plan
@@ -1,24 +1,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$122(ASC), $$123(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$122(ASC), $$123(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- HYBRID_HASH_JOIN [$$122][$$123]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$128(ASC), $$129(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$128(ASC), $$129(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$128][$$129]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$128]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.ds4)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.ds4)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$123]  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$129]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.ds1)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.ds1)  |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/btree-index-selection/cast-default-null/cast-default-null-09.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-09.plan
index d60ee1e..d162de1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-09.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-09.plan
@@ -1,24 +1,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$84(ASC), $$85(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$84(ASC), $$85(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- HYBRID_HASH_JOIN [$$84][$$85]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$84]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$89(ASC), $$90(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$89(ASC), $$90(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$89][$$90]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$89]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.ds4)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.ds4)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$90]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.ds2)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.ds2)  |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/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 16f44e8..588a92a 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
@@ -1,21 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- 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_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$50(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (test.ds5.ds5)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (test.ds5.ds5)  |PARTITIONED|
+                      -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- BTREE_SEARCH (test.ds5.idx5_dt_fmt)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.ds5.idx5_dt_fmt)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |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 297794d..918da4d 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
@@ -1,21 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- 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_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$50(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (test.ds5.ds5)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (test.ds5.ds5)  |PARTITIONED|
+                      -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- BTREE_SEARCH (test.ds5.idx5_d_fmt)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.ds5.idx5_d_fmt)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |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 910a579..372ef30 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
@@ -1,21 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- 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_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$50(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (test.ds5.ds5)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (test.ds5.ds5)  |PARTITIONED|
+                      -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- BTREE_SEARCH (test.ds5.idx5_t_fmt)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.ds5.idx5_t_fmt)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |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 3cc15fb..f03306d 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
@@ -1,21 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- 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_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$50(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (test.ds6.ds6)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (test.ds6.ds6)  |PARTITIONED|
+                      -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- BTREE_SEARCH (test.ds6.idx6_dt)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.ds6.idx6_dt)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |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 fdd7255..40197ad 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
@@ -1,21 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- 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_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$50(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (test.ds6.ds6)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (test.ds6.ds6)  |PARTITIONED|
+                      -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- BTREE_SEARCH (test.ds6.idx6_d)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.ds6.idx6_d)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |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 3b5b684..9f5ad1b 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
@@ -1,21 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- 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_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$50(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (test.ds6.ds6)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (test.ds6.ds6)  |PARTITIONED|
+                      -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- BTREE_SEARCH (test.ds6.idx6_t)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.ds6.idx6_t)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |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 62d8cf1..9e04d8e 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
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- 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_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$50(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.ds5)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.ds5)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |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 607e59c..c488edd 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
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- 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_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$50(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.ds6)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.ds6)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |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 62d8cf1..9e04d8e 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
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- 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_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$50(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.ds5)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.ds5)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |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-19.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-19.plan
index 2327f39..1587d66 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-19.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-19.plan
@@ -1,27 +1,24 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$110(ASC), $$111(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$110(ASC), $$111(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$116(ASC), $$117(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$116(ASC), $$117(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (test.ds5.ds5)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (test.ds5.ds5)  |PARTITIONED|
+                        -- STABLE_SORT [$$129(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$123(ASC)]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (test.ds5.idx5_dt_fmt)  |PARTITIONED|
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN (test.ds6)  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.ds5.idx5_dt_fmt)  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |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-20.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-20.plan
index 62dc0e9..7375151 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
@@ -1,21 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$60(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (test.ds7.ds7)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (test.ds7.ds7)  |PARTITIONED|
+                      -- STABLE_SORT [$$72(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$72(ASC)]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- BTREE_SEARCH (test.ds7.idx2)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.ds7.idx2)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |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 37bd6b8..161c4d4 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
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$60(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.ds7)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.ds7)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |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 37bd6b8..161c4d4 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
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$60(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.ds7)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.ds7)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |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 9d2902b..1270c47 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
@@ -1,21 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$60(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (test.ds7.ds7)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (test.ds7.ds7)  |PARTITIONED|
+                      -- STABLE_SORT [$$72(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$72(ASC)]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- BTREE_SEARCH (test.ds7.idx3)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.ds7.idx3)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |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 d510067..fa7d6db 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
@@ -1,28 +1,15 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- UNION_ALL  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.ds7.ds7)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- SPLIT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.ds7.idx1)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- UNION_ALL  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.ds7.ds7)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- SPLIT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -30,3 +17,13 @@
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- 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 62dc0e9..7375151 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
@@ -1,21 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$60(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (test.ds7.ds7)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (test.ds7.ds7)  |PARTITIONED|
+                      -- STABLE_SORT [$$72(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$72(ASC)]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- BTREE_SEARCH (test.ds7.idx2)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.ds7.idx2)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |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-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/disjunctive-predicate/disjunctive-predicate-2.plan
index 7f726b6..642109c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/disjunctive-predicate/disjunctive-predicate-2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/disjunctive-predicate/disjunctive-predicate-2.plan
@@ -1,16 +1,15 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- BTREE_SEARCH (test.TestSet.TestSet)  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (test.TestSet.TestSetIndex)  |PARTITIONED|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- UNNEST  |UNPARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- BTREE_SEARCH (test.TestSet.TestSet)  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.TestSet.TestSetIndex)  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- UNNEST  |UNPARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/disjunctive-predicate/disjunctive-predicate-4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/disjunctive-predicate/disjunctive-predicate-4.plan
index dcffc94..2db6d1a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/disjunctive-predicate/disjunctive-predicate-4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/disjunctive-predicate/disjunctive-predicate-4.plan
@@ -1,11 +1,10 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
         -- ASSIGN  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- DATASOURCE_SCAN (test.TestSet)  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN (test.TestSet)  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |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 7f72a7e..cfc9617 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
@@ -3,10 +3,9 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       -- 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|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (test.TestSet)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/function-on-pk/function-on-pk-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/function-on-pk/function-on-pk-01.plan
index f621777..9f4d55c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/function-on-pk/function-on-pk-01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/function-on-pk/function-on-pk-01.plan
@@ -1,10 +1,8 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- DATASOURCE_SCAN (test.ds1)  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- DATASOURCE_SCAN (test.ds1)  |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/btree-index-selection/function-on-pk/function-on-pk-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/function-on-pk/function-on-pk-02.plan
index f621777..9f4d55c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/function-on-pk/function-on-pk-02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/function-on-pk/function-on-pk-02.plan
@@ -1,10 +1,8 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- DATASOURCE_SCAN (test.ds1)  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- DATASOURCE_SCAN (test.ds1)  |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/btree-index-selection/hints-skip-index/hints-skip-index-10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-10.plan
index 7af465d..cfff6e8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-10.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-10.plan
@@ -1,19 +1,18 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$19(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |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/hints-skip-index/hints-skip-index-11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-11.plan
index 51bb0f2..88609bd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-11.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-11.plan
@@ -1,18 +1,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$20][$$22]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$21][$$23]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
               -- BROADCAST_EXCHANGE  |PARTITIONED|
                 -- UNNEST  |UNPARTITIONED|
-                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
\ No newline at end of file
+                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-12.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-12.plan
index 55140ef..82a6ab3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-12.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-12.plan
@@ -1,20 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
-                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                  -- ASSIGN  |UNPARTITIONED|
-                                    -- UNNEST  |UNPARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
\ No newline at end of file
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |UNPARTITIONED|
+                                  -- UNNEST  |UNPARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-13.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-13.plan
index 102a5e7..1fe8eeb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-13.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-13.plan
@@ -1,19 +1,18 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$19(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.tenk.idx_1k)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.tenk.idx_1k)  |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/hints-skip-index/hints-skip-index-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-3.plan
index f958053..da99423 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-3.plan
@@ -1,12 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$19(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-4.plan
index 7af465d..cfff6e8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-4.plan
@@ -1,19 +1,18 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$19(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |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/hints-skip-index/hints-skip-index-5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-5.plan
index f958053..da99423 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-5.plan
@@ -1,12 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$19(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-6.plan
index 5a83b61..106ce2c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-6.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-6.plan
@@ -1,19 +1,18 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.tenk.idx_2k)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.tenk.idx_2k)  |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/hints-skip-index/hints-skip-index-7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-7.plan
index a84e5c8..a28e2c4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-7.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-7.plan
@@ -1,30 +1,29 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INTERSECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.tenk.idx_2k)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- INTERSECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.idx_2k)  |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/hints-skip-index/hints-skip-index-8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-8.plan
index 1d31f3b..797e5fa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-8.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-8.plan
@@ -1,19 +1,18 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |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/hints-skip-index/hints-skip-index-9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-9.plan
index 102a5e7..1fe8eeb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-9.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-skip-index/hints-skip-index-9.plan
@@ -1,19 +1,18 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$19(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.tenk.idx_1k)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.tenk.idx_1k)  |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/hints-use-index/hints-use-index-10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-10.plan
index 83ba8b8..1fe8eeb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-10.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-10.plan
@@ -2,18 +2,17 @@
   -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
     -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.tenk.idx_1k)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.tenk.idx_1k)  |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/hints-use-index/hints-use-index-11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-11.plan
index 83ba8b8..1fe8eeb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-11.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-11.plan
@@ -2,18 +2,17 @@
   -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
     -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.tenk.idx_1k)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.tenk.idx_1k)  |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/hints-use-index/hints-use-index-12.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-12.plan
index ed6dbe8..b571087 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-12.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-12.plan
@@ -2,36 +2,34 @@
   -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
     -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INTERSECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.tenk.idx_1k)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- INTERSECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.idx_1k)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.tenk.idx_5k)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- REPLICATE  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.idx_5k)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |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/hints-use-index/hints-use-index-13.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-13.plan
index 83ba8b8..1fe8eeb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-13.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-13.plan
@@ -2,18 +2,17 @@
   -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
     -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.tenk.idx_1k)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.tenk.idx_1k)  |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/hints-use-index/hints-use-index-14.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-14.plan
index 6eb942d..07438f8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-14.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-14.plan
@@ -2,18 +2,17 @@
   -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
     -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.tenk.idx_2k)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.tenk.idx_2k)  |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/hints-use-index/hints-use-index-15.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-15.plan
index 725e81e..106ce2c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-15.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-15.plan
@@ -2,18 +2,17 @@
   -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
     -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.tenk.idx_2k)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.tenk.idx_2k)  |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/hints-use-index/hints-use-index-16.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-16.plan
index cf09754..9a52414 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-16.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-16.plan
@@ -2,36 +2,34 @@
   -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
     -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INTERSECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.tenk.idx_1k)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- INTERSECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.idx_1k)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.tenk.idx_2k)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- REPLICATE  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.idx_2k)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |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/hints-use-index/hints-use-index-17.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-17.plan
index f04dd37..ad64bb8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-17.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-17.plan
@@ -1,19 +1,18 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |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/hints-use-index/hints-use-index-18.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-18.plan
index 102a5e7..1fe8eeb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-18.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-18.plan
@@ -1,19 +1,18 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$19(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.tenk.idx_1k)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.tenk.idx_1k)  |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/hints-use-index/hints-use-index-19.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-19.plan
index 102a5e7..1fe8eeb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-19.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-19.plan
@@ -1,19 +1,18 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$19(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.tenk.idx_1k)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.tenk.idx_1k)  |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/hints-use-index/hints-use-index-20.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-20.plan
index c226460..aa0bf62 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-20.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-20.plan
@@ -1,28 +1,24 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- UNNEST  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- UNNEST  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.orders.orders)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (test.orders.orders)  |PARTITIONED|
+                                -- STABLE_SORT [$$59(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STABLE_SORT [$$56(ASC)]  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.orders.orderline_delivery_d)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- BTREE_SEARCH (test.orders.orderline_delivery_d)  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-21.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-21.plan
index 5bde8ba..d018905 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-21.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-21.plan
@@ -1,28 +1,24 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- UNNEST  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- UNNEST  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.orders.orders)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (test.orders.orders)  |PARTITIONED|
+                                -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STABLE_SORT [$$55(ASC)]  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.orders.orderline_delivery_d)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- BTREE_SEARCH (test.orders.orderline_delivery_d)  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-3.plan
index 7af465d..cfff6e8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-3.plan
@@ -1,19 +1,18 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$19(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |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/hints-use-index/hints-use-index-4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-4.plan
index ce4ed8e..9a52414 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-4.plan
@@ -1,37 +1,35 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INTERSECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.tenk.idx_1k)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- INTERSECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.idx_1k)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.tenk.idx_2k)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- REPLICATE  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.idx_2k)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |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/hints-use-index/hints-use-index-5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-5.plan
index d68a5a1..5e57a22 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-5.plan
@@ -1,37 +1,35 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INTERSECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- INTERSECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.tenk.idx_5k_10k)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- REPLICATE  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.idx_5k_10k)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |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/hints-use-index/hints-use-index-6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-6.plan
index 1d31f3b..797e5fa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-6.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-6.plan
@@ -1,19 +1,18 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |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/hints-use-index/hints-use-index-7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-7.plan
index 1d31f3b..797e5fa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-7.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-7.plan
@@ -1,19 +1,18 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |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/hints-use-index/hints-use-index-8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-8.plan
index 8594229..e32ac40 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-8.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-8.plan
@@ -1,30 +1,29 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INTERSECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.tenk.idx_2k_5k)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- INTERSECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.idx_2k_5k)  |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/hints-use-index/hints-use-index-9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-9.plan
index 8594229..e32ac40 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-9.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/hints-use-index/hints-use-index-9.plan
@@ -1,30 +1,29 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INTERSECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.tenk.idx_2k_5k)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- INTERSECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.idx_2k_5k)  |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/intersection-misc/intersection-misc-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/intersection-misc/intersection-misc-01.plan
index 774c135..98dcddd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/intersection-misc/intersection-misc-01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/intersection-misc/intersection-misc-01.plan
@@ -1,128 +1,119 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$$135(ASC), $$125(ASC), $$137(ASC) ]  |PARTITIONED|
-            -- STABLE_SORT [$$135(ASC), $$125(ASC), $$137(ASC)]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- HYBRID_HASH_JOIN [$$130][$$137]  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$135][$$129]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$142(ASC), $$132(ASC), $$144(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$142(ASC), $$132(ASC), $$144(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$137][$$144]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$142][$$136]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$142]  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
-                                    -- STREAM_SELECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- PRE_CLUSTERED_GROUP_BY[$$130]  |PARTITIONED|
+                                              {
+                                                -- AGGREGATE  |LOCAL|
+                                                  -- STREAM_SELECT  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                              }
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STABLE_SORT [$$130(ASC)]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- HYBRID_HASH_JOIN [$$139][$$85]  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$139]  |PARTITIONED|
+                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- BTREE_SEARCH (test.d.d)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- REPLICATE  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- STABLE_SORT [$$155(ASC)]  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- BTREE_SEARCH (test.d.idx_c5)  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$136]  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$144]  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$133]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$133(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- PRE_CLUSTERED_GROUP_BY[$$123]  |PARTITIONED|
-                                                  {
-                                                    -- AGGREGATE  |LOCAL|
-                                                      -- STREAM_SELECT  |LOCAL|
-                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                  }
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STABLE_SORT [$$123(ASC)]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$123]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- HYBRID_HASH_JOIN [$$132][$$78]  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$132]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- STREAM_SELECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- BTREE_SEARCH (test.d.d)  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- REPLICATE  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- STABLE_SORT [$$148(ASC)]  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- BTREE_SEARCH (test.d.idx_c5)  |PARTITIONED|
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- REPLICATE  |PARTITIONED|
-                                                            -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$129]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$126]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$126(ASC)]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$126]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$133][$$97]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- BTREE_SEARCH (test.d.d)  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- INTERSECT  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ASSIGN  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- REPLICATE  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- STABLE_SORT [$$148(ASC)]  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    -- BTREE_SEARCH (test.d.idx_c5)  |PARTITIONED|
-                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- ASSIGN  |PARTITIONED|
-                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- STABLE_SORT [$$156(ASC)]  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- BTREE_SEARCH (test.d.idx_c6)  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$140][$$104]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$140]  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH (test.d.d)  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- REPLICATE  |PARTITIONED|
-                                                          -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
+                                                        -- INTERSECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- REPLICATE  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- STABLE_SORT [$$155(ASC)]  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- BTREE_SEARCH (test.d.idx_c5)  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STABLE_SORT [$$163(ASC)]  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
+                                                                    -- BTREE_SEARCH (test.d.idx_c6)  |PARTITIONED|
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- REPLICATE  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/intersection-misc/intersection-misc-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/intersection-misc/intersection-misc-02.plan
index 81f2d40..a4d6a43 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/intersection-misc/intersection-misc-02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/intersection-misc/intersection-misc-02.plan
@@ -1,116 +1,107 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$$135(ASC), $$125(ASC), $$137(ASC) ]  |PARTITIONED|
-            -- STABLE_SORT [$$135(ASC), $$125(ASC), $$137(ASC)]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- HYBRID_HASH_JOIN [$$130][$$137]  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$135][$$129]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$142(ASC), $$132(ASC), $$144(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$142(ASC), $$132(ASC), $$144(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$137][$$144]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$142][$$136]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$142]  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
-                                    -- STREAM_SELECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- PRE_CLUSTERED_GROUP_BY[$$130]  |PARTITIONED|
+                                              {
+                                                -- AGGREGATE  |LOCAL|
+                                                  -- STREAM_SELECT  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                              }
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STABLE_SORT [$$130(ASC)]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- HYBRID_HASH_JOIN [$$139][$$85]  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$139]  |PARTITIONED|
+                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- REPLICATE  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- BTREE_SEARCH (test.d.d)  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- STABLE_SORT [$$155(ASC)]  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- BTREE_SEARCH (test.d.idx_c5)  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$136]  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$144]  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$133]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$133(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- PRE_CLUSTERED_GROUP_BY[$$123]  |PARTITIONED|
-                                                  {
-                                                    -- AGGREGATE  |LOCAL|
-                                                      -- STREAM_SELECT  |LOCAL|
-                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                  }
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STABLE_SORT [$$123(ASC)]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$123]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$140][$$104]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$140]  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- HYBRID_HASH_JOIN [$$132][$$78]  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$132]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- STREAM_SELECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- REPLICATE  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- BTREE_SEARCH (test.d.d)  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- STABLE_SORT [$$148(ASC)]  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- BTREE_SEARCH (test.d.idx_c5)  |PARTITIONED|
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- REPLICATE  |PARTITIONED|
-                                                            -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
+                                                          -- BTREE_SEARCH (test.d.d)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- STABLE_SORT [$$155(ASC)]  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                   -- STREAM_PROJECT  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
+                                                                      -- BTREE_SEARCH (test.d.idx_c5)  |PARTITIONED|
                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$129]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$126]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$126(ASC)]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$126]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$133][$$97]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- STREAM_SELECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- REPLICATE  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
                                                       -- ASSIGN  |PARTITIONED|
                                                         -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- REPLICATE  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- BTREE_SEARCH (test.d.d)  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- STABLE_SORT [$$148(ASC)]  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- BTREE_SEARCH (test.d.idx_c5)  |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 [$$78]  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/intersection-misc/intersection-misc-03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/intersection-misc/intersection-misc-03.plan
index 0145b58..fb235a5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/intersection-misc/intersection-misc-03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/intersection-misc/intersection-misc-03.plan
@@ -1,110 +1,102 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$$135(ASC), $$125(ASC), $$137(ASC) ]  |PARTITIONED|
-            -- STABLE_SORT [$$135(ASC), $$125(ASC), $$137(ASC)]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- HYBRID_HASH_JOIN [$$130][$$137]  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$135][$$129]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$142(ASC), $$132(ASC), $$144(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$142(ASC), $$132(ASC), $$144(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$137][$$144]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$142][$$136]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$142]  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
-                                    -- STREAM_SELECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- PRE_CLUSTERED_GROUP_BY[$$130]  |PARTITIONED|
+                                              {
+                                                -- AGGREGATE  |LOCAL|
+                                                  -- STREAM_SELECT  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                              }
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STABLE_SORT [$$130(ASC)]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- HYBRID_HASH_JOIN [$$139][$$85]  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$139]  |PARTITIONED|
+                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- BTREE_SEARCH (test.d.d)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STABLE_SORT [$$155(ASC)]  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- BTREE_SEARCH (test.d.idx_c5)  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$136]  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$144]  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$133]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$133(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- PRE_CLUSTERED_GROUP_BY[$$123]  |PARTITIONED|
-                                                  {
-                                                    -- AGGREGATE  |LOCAL|
-                                                      -- STREAM_SELECT  |LOCAL|
-                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                  }
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STABLE_SORT [$$123(ASC)]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$123]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- HYBRID_HASH_JOIN [$$132][$$78]  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$132]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- STREAM_SELECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- BTREE_SEARCH (test.d.d)  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- STABLE_SORT [$$148(ASC)]  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- BTREE_SEARCH (test.d.idx_c5)  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- REPLICATE  |PARTITIONED|
-                                                            -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$129]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$126]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$126(ASC)]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$126]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$133][$$97]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- BTREE_SEARCH (test.d.d)  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- STABLE_SORT [$$152(ASC)]  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- BTREE_SEARCH (test.d.idx_c6)  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$140][$$104]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$140]  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH (test.d.d)  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- REPLICATE  |PARTITIONED|
-                                                          -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
+                                                        -- STABLE_SORT [$$159(ASC)]  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                             -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- BTREE_SEARCH (test.d.idx_c6)  |PARTITIONED|
                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- REPLICATE  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/intersection-misc/intersection-misc-04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/intersection-misc/intersection-misc-04.plan
index d3cde62..33ccc2e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/intersection-misc/intersection-misc-04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/intersection-misc/intersection-misc-04.plan
@@ -1,106 +1,96 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$$135(ASC), $$125(ASC), $$137(ASC) ]  |PARTITIONED|
-            -- STABLE_SORT [$$135(ASC), $$125(ASC), $$137(ASC)]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- HYBRID_HASH_JOIN [$$130][$$137]  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$135][$$129]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$142(ASC), $$132(ASC), $$144(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$142(ASC), $$132(ASC), $$144(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$137][$$144]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$142][$$136]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$142]  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
-                                    -- STREAM_SELECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- PRE_CLUSTERED_GROUP_BY[$$130]  |PARTITIONED|
+                                              {
+                                                -- AGGREGATE  |LOCAL|
+                                                  -- STREAM_SELECT  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                              }
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STABLE_SORT [$$130(ASC)]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- HYBRID_HASH_JOIN [$$139][$$85]  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$139]  |PARTITIONED|
+                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- REPLICATE  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$136]  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$144]  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$133]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$133(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- PRE_CLUSTERED_GROUP_BY[$$123]  |PARTITIONED|
-                                                  {
-                                                    -- AGGREGATE  |LOCAL|
-                                                      -- STREAM_SELECT  |LOCAL|
-                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                  }
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STABLE_SORT [$$123(ASC)]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$123]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$140][$$104]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$140]  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- HYBRID_HASH_JOIN [$$132][$$78]  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$132]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- STREAM_SELECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- REPLICATE  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- REPLICATE  |PARTITIONED|
-                                                            -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$129]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- REPLICATE  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$126]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$126(ASC)]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$126]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$133][$$97]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- STREAM_SELECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- REPLICATE  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
                                                       -- ASSIGN  |PARTITIONED|
                                                         -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- REPLICATE  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (test.d)  |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 [$$78]  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-47.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-47.plan
index e9f5574..95526b9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-47.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-47.plan
@@ -1,17 +1,16 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- BTREE_SEARCH (test.testdst.testdst)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.testdst.sec_Idx)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.testdst.testdst)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.testdst.sec_Idx)  |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/btree-secondary-48.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-48.plan
index e9f5574..95526b9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-48.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-48.plan
@@ -1,17 +1,16 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- BTREE_SEARCH (test.testdst.testdst)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.testdst.sec_Idx)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.testdst.testdst)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.testdst.sec_Idx)  |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/btree-secondary-51.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan
index e9f5574..95526b9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan
@@ -1,17 +1,16 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- BTREE_SEARCH (test.testdst.testdst)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.testdst.sec_Idx)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.testdst.testdst)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.testdst.sec_Idx)  |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/btree-secondary-52.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan
index e9f5574..95526b9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan
@@ -1,17 +1,16 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- BTREE_SEARCH (test.testdst.testdst)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.testdst.sec_Idx)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.testdst.testdst)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.testdst.sec_Idx)  |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/btree-secondary-53.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan
index e9f5574..95526b9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan
@@ -1,17 +1,16 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- BTREE_SEARCH (test.testdst.testdst)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.testdst.sec_Idx)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.testdst.testdst)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.testdst.sec_Idx)  |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/btree-secondary-58.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan
index c311b11..218a98c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan
@@ -1,17 +1,16 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- BTREE_SEARCH (test.testdst.testdst)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.testdst.sec_Idx)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.testdst.testdst)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.testdst.sec_Idx)  |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/btree-secondary-59.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-59.plan
index c311b11..218a98c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-59.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-59.plan
@@ -1,17 +1,16 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- BTREE_SEARCH (test.testdst.testdst)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.testdst.sec_Idx)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.testdst.testdst)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.testdst.sec_Idx)  |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/btree-secondary-61.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-61.plan
index e9f5574..95526b9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-61.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-61.plan
@@ -1,17 +1,16 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- BTREE_SEARCH (test.testdst.testdst)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.testdst.sec_Idx)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.testdst.testdst)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.testdst.sec_Idx)  |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/btree-secondary-68.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-68.plan
index 11f5d9f..25f3a1f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-68.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-68.plan
@@ -1,32 +1,30 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$27(ASC), $$28(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$27(ASC), $$28(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$28(ASC), $$29(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$28(ASC), $$29(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
+                      -- INTERSECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- INTERSECT  |PARTITIONED|
+                          -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$37(ASC)]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- BTREE_SEARCH (tpch.Orders.idx_custkey)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (tpch.Orders.idx_custkey)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$41(ASC)]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- BTREE_SEARCH (tpch.Orders.idx_orderpriority)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (tpch.Orders.idx_orderpriority)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-68_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-68_ps.plan
index c135758..d9ccf43 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-68_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-68_ps.plan
@@ -1,69 +1,66 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$27(ASC), $$28(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$27(ASC), $$28(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$28(ASC), $$29(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$28(ASC), $$29(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
+                              -- INTERSECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- INTERSECT  |PARTITIONED|
+                                  -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$37(ASC)]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                          -- BTREE_SEARCH (tpch.Orders.idx_custkey)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- BTREE_SEARCH (tpch.Orders.idx_custkey)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$41(ASC)]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                          -- BTREE_SEARCH (tpch.Orders.idx_orderpriority)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- BTREE_SEARCH (tpch.Orders.idx_orderpriority)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
+                                        -- INTERSECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- INTERSECT  |PARTITIONED|
+                                            -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STABLE_SORT [$$37(ASC)]  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- BTREE_SEARCH (tpch.Orders.idx_custkey)  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- BTREE_SEARCH (tpch.Orders.idx_custkey)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STABLE_SORT [$$41(ASC)]  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- BTREE_SEARCH (tpch.Orders.idx_orderpriority)  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- BTREE_SEARCH (tpch.Orders.idx_orderpriority)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |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 f37ce33..f70a511 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
@@ -1,21 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (test.MyData.MyData)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (test.MyData.MyData)  |PARTITIONED|
+                      -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- BTREE_SEARCH (test.MyData.btree_index_docid)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.MyData.btree_index_docid)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-01-disable-idxonly_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-01-disable-idxonly_ps.plan
index 6fbe462..7852236 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-01-disable-idxonly_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-01-disable-idxonly_ps.plan
@@ -1,47 +1,44 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$20(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$21(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (test.MyData.MyData)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.MyData.MyData)  |PARTITIONED|
+                              -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (test.MyData.btree_index_docid)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (test.MyData.btree_index_docid)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.MyData.MyData)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.MyData.MyData)  |PARTITIONED|
+                                        -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                -- BTREE_SEARCH (test.MyData.btree_index_docid)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (test.MyData.btree_index_docid)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-01.plan
index 9465d59..82888fd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-01.plan
@@ -1,28 +1,15 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- UNION_ALL  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.MyData.MyData)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- SPLIT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.MyData.btree_index_docid)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- UNION_ALL  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.MyData.MyData)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- SPLIT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -30,3 +17,13 @@
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |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/btree-index/btree-sidx-idxonly-01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-01_ps.plan
index 70d8a30..5e00152 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-01_ps.plan
@@ -1,32 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$21(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- UNION_ALL  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_SELECT  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.MyData.MyData)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- SPLIT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (test.MyData.btree_index_docid)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$21(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- UNION_ALL  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH (test.MyData.MyData)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- SPLIT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -34,32 +21,30 @@
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- ASSIGN  |PARTITIONED|
                                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- UNION_ALL  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_SELECT  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (test.MyData.MyData)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- SPLIT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (test.MyData.btree_index_docid)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- STREAM_SELECT  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- SPLIT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (test.MyData.btree_index_docid)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- UNION_ALL  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.MyData.MyData)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- SPLIT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -67,3 +52,13 @@
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                     -- ASSIGN  |PARTITIONED|
                                                       -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- SPLIT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |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/btree-index/btree-sidx-idxonly-10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-10.plan
index 2dcfdd2..794a51d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-10.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-10.plan
@@ -6,30 +6,26 @@
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- UNION_ALL  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (twitter.ds_tweet.ds_tweet)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- SPLIT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (twitter.ds_tweet.create_at_status_count_idx)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (twitter.ds_tweet.ds_tweet)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- SPLIT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (twitter.ds_tweet.create_at_status_count_idx)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- SPLIT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (twitter.ds_tweet.create_at_status_count_idx)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (twitter.ds_tweet.create_at_status_count_idx)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query1.plan
index 671d511..82658f0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query1.plan
@@ -1,35 +1,33 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH (TestDataverse.IndexDatasetB.IndexDatasetB)  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STABLE_SORT [$$73(ASC)]  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (TestDataverse.IndexDatasetB.indexB)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestDataverse.IndexDatasetB.IndexDatasetB)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$77(ASC)]  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH (TestDataverse.IndexDatasetB.indexB)  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
-                                      -- STREAM_SELECT  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (TestDataverse.IndexDatasetA.IndexDatasetA)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (TestDataverse.IndexDatasetA.IndexDatasetA)  |PARTITIONED|
+                                            -- STABLE_SORT [$$75(ASC)]  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STABLE_SORT [$$71(ASC)]  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- BTREE_SEARCH (TestDataverse.IndexDatasetA.indexA)  |PARTITIONED|
-                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- BTREE_SEARCH (TestDataverse.IndexDatasetA.indexA)  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (TestDataverse.ProbeDataset)  |PARTITIONED|
                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- DATASOURCE_SCAN (TestDataverse.ProbeDataset)  |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/btree-ternary-inlj/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query2.plan
index c0f794e..42c4f34 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query2.plan
@@ -1,37 +1,34 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH (TestDataverse.IndexDatasetB.IndexDatasetB)  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STABLE_SORT [$$73(ASC)]  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (TestDataverse.IndexDatasetB.indexB)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestDataverse.IndexDatasetB.IndexDatasetB)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$77(ASC)]  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH (TestDataverse.IndexDatasetB.indexB)  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
-                                      -- STREAM_SELECT  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (TestDataverse.IndexDatasetA.IndexDatasetA)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (TestDataverse.IndexDatasetA.IndexDatasetA)  |PARTITIONED|
+                                            -- STABLE_SORT [$$75(ASC)]  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STABLE_SORT [$$71(ASC)]  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- BTREE_SEARCH (TestDataverse.IndexDatasetA.indexA)  |PARTITIONED|
-                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (TestDataverse.ProbeDataset)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                                    -- BTREE_SEARCH (TestDataverse.IndexDatasetA.indexA)  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN (TestDataverse.ProbeDataset)  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query3.plan
index b94c26d..1bb7cee 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query3.plan
@@ -1,49 +1,45 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH (TestDataverse.IndexDatasetC.IndexDatasetC)  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STABLE_SORT [$$91(ASC)]  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (TestDataverse.IndexDatasetC.indexC)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestDataverse.IndexDatasetC.IndexDatasetC)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$96(ASC)]  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH (TestDataverse.IndexDatasetC.indexC)  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
-                                      -- STREAM_SELECT  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (TestDataverse.IndexDatasetB.IndexDatasetB)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (TestDataverse.IndexDatasetB.IndexDatasetB)  |PARTITIONED|
+                                            -- STABLE_SORT [$$94(ASC)]  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STABLE_SORT [$$89(ASC)]  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- BTREE_SEARCH (TestDataverse.IndexDatasetB.indexB)  |PARTITIONED|
-                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- STREAM_SELECT  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- BTREE_SEARCH (TestDataverse.IndexDatasetA.IndexDatasetA)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- STABLE_SORT [$$87(ASC)]  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- BTREE_SEARCH (TestDataverse.IndexDatasetA.indexA)  |PARTITIONED|
-                                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            -- DATASOURCE_SCAN (TestDataverse.ProbeDataset)  |PARTITIONED|
-                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                                    -- BTREE_SEARCH (TestDataverse.IndexDatasetB.indexB)  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH (TestDataverse.IndexDatasetA.IndexDatasetA)  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- STABLE_SORT [$$92(ASC)]  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- BTREE_SEARCH (TestDataverse.IndexDatasetA.indexA)  |PARTITIONED|
+                                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- DATASOURCE_SCAN (TestDataverse.ProbeDataset)  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query4.plan
index 4e1a931..2018224 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query4.plan
@@ -1,32 +1,30 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- HYBRID_HASH_JOIN [$$57][$$70]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (tpcds.customer_address.customer_address)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$74(ASC)]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (tpcds.customer_demographics.customer_demographics)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STABLE_SORT [$$72(ASC)]  |PARTITIONED|
-                                          -- HASH_PARTITION_EXCHANGE [$$72]  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- HYBRID_HASH_JOIN [$$61][$$74]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (tpcds.customer_address.customer_address)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$78(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (tpcds.customer_demographics.customer_demographics)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$76(ASC)]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$76]  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
                                             -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (tpcds.customer)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (tpcds.customer)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- UNNEST  |UNPARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- UNNEST  |UNPARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q1.plan
index 5eee969..8fe17e7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q1.plan
@@ -1,29 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$ol_number(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$107]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$107]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$92]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$ol_number(ASC) ]  |PARTITIONED|
+        -- SORT_GROUP_BY[$$111]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$111]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$96]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- UNNEST  |PARTITIONED|
                       -- ASSIGN  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
-                          -- UNNEST  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |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/ch2/ch2_q10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q10.plan
index 7cc19d7..df289e3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q10.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q10.plan
@@ -1,59 +1,52 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     -- STREAM_LIMIT  |UNPARTITIONED|
-      -- STREAM_PROJECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$$164(DESC) ]  |PARTITIONED|
-            -- STREAM_LIMIT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [topK: 20] [$$164(DESC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- SORT_GROUP_BY[$$171, $$172, $$173, $$174, $$175]  |PARTITIONED|
-                            {
-                              -- AGGREGATE  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                            }
-                      -- HASH_PARTITION_EXCHANGE [$$171, $$172, $$173, $$174, $$175]  |PARTITIONED|
-                        -- SORT_GROUP_BY[$$155, $$150, $$151, $$152, $$153]  |PARTITIONED|
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- HYBRID_HASH_JOIN [$$155, $$160, $$162][$$159, $$161, $$163]  |PARTITIONED|
-                                  -- HASH_PARTITION_EXCHANGE [$$155, $$160, $$162]  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- HYBRID_HASH_JOIN [$$165][$$166]  |PARTITIONED|
-                                          -- HASH_PARTITION_EXCHANGE [$$165]  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$175(DESC) ]  |PARTITIONED|
+          -- STREAM_LIMIT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [topK: 20] [$$175(DESC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- SORT_GROUP_BY[$$182, $$183, $$184, $$185, $$186]  |PARTITIONED|
+                          {
+                            -- AGGREGATE  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                          }
+                    -- HASH_PARTITION_EXCHANGE [$$182, $$183, $$184, $$185, $$186]  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$166, $$161, $$162, $$163, $$164]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$166, $$171, $$173][$$170, $$172, $$174]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$166, $$171, $$173]  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- HYBRID_HASH_JOIN [$$176][$$177]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$176]  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
                                             -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                          -- HASH_PARTITION_EXCHANGE [$$166]  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$177]  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
                                             -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                  -- HASH_PARTITION_EXCHANGE [$$159, $$161, $$163]  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- UNNEST  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$170, $$172, $$174]  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- UNNEST  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |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/ch2/ch2_q11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q11.plan
index 5133ed2..4ef2cfe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q11.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q11.plan
@@ -1,62 +1,100 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$178(DESC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$178(DESC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- NESTED_LOOP  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- SORT_GROUP_BY[$$204]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- HASH_PARTITION_EXCHANGE [$$204]  |PARTITIONED|
-                          -- SORT_GROUP_BY[$$168]  |PARTITIONED|
-                                  {
-                                    -- AGGREGATE  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                  }
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$176][$$175]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$176]  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- HYBRID_HASH_JOIN [$$188][$$187]  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- REPLICATE  |PARTITIONED|
-                                                      -- HASH_PARTITION_EXCHANGE [$$193]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$188(DESC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$188(DESC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- SORT_GROUP_BY[$$214]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- HASH_PARTITION_EXCHANGE [$$214]  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$178]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$186][$$185]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$186]  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$198][$$197]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- REPLICATE  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$203]  |PARTITIONED|
+                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
                                                         -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- STREAM_SELECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (test.nation)  |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 [$$192]  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$175]  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- REPLICATE  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$202]  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$185]  |PARTITIONED|
+                                    -- ASSIGN  |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 (test.stock)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- ASSIGN  |UNPARTITIONED|
+                      -- AGGREGATE  |UNPARTITIONED|
+                        -- ASSIGN  |UNPARTITIONED|
+                          -- AGGREGATE  |UNPARTITIONED|
+                            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                              -- AGGREGATE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$201][$$200]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$201]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$203][$$202]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- REPLICATE  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$203]  |PARTITIONED|
+                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- REPLICATE  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$202]  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$200]  |PARTITIONED|
                                         -- ASSIGN  |PARTITIONED|
                                           -- ASSIGN  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -67,53 +105,3 @@
                                                       -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |UNPARTITIONED|
-                        -- ASSIGN  |UNPARTITIONED|
-                          -- AGGREGATE  |UNPARTITIONED|
-                            -- STREAM_PROJECT  |UNPARTITIONED|
-                              -- ASSIGN  |UNPARTITIONED|
-                                -- AGGREGATE  |UNPARTITIONED|
-                                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                    -- AGGREGATE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- HYBRID_HASH_JOIN [$$191][$$190]  |PARTITIONED|
-                                            -- HASH_PARTITION_EXCHANGE [$$191]  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- HYBRID_HASH_JOIN [$$193][$$192]  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- REPLICATE  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$193]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- STREAM_SELECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- REPLICATE  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$192]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                            -- HASH_PARTITION_EXCHANGE [$$190]  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |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 (test.stock)  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q12.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q12.plan
index 1c561d0..dfb7cbc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q12.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q12.plan
@@ -1,29 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$o_ol_cnt(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$96]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$96]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$85]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- UNNEST  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$o_ol_cnt(ASC) ]  |PARTITIONED|
+        -- SORT_GROUP_BY[$$100]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$100]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$89]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- UNNEST  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |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/ch2/ch2_q13.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q13.plan
index b33dc89..7588ece 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q13.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q13.plan
@@ -1,53 +1,50 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$125(DESC), $$c_count(DESC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$125(DESC), $$c_count(DESC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- SORT_GROUP_BY[$$139]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- HASH_PARTITION_EXCHANGE [$$139]  |PARTITIONED|
-                  -- SORT_GROUP_BY[$$124]  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- SORT_GROUP_BY[$$137]  |PARTITIONED|
-                                  {
-                                    -- AGGREGATE  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                  }
-                            -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
-                              -- SORT_GROUP_BY[$$126]  |PARTITIONED|
-                                      {
-                                        -- AGGREGATE  |LOCAL|
-                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                      }
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- HYBRID_HASH_JOIN [$$126, $$128, $$130][$$127, $$129, $$131]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$126, $$128, $$130]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$133(DESC), $$c_count(DESC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$133(DESC), $$c_count(DESC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- SORT_GROUP_BY[$$147]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- HASH_PARTITION_EXCHANGE [$$147]  |PARTITIONED|
+                -- SORT_GROUP_BY[$$132]  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$145]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- HASH_PARTITION_EXCHANGE [$$145]  |PARTITIONED|
+                            -- SORT_GROUP_BY[$$134]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$134, $$136, $$138][$$135, $$137, $$139]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$134, $$136, $$138]  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$127, $$129, $$131]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- STREAM_SELECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$135, $$137, $$139]  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q14.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q14.plan
index 6dda602..102418b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q14.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q14.plan
@@ -1,34 +1,27 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
               -- STREAM_PROJECT  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$84][$$83]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$84]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$93][$$92]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$93]  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$92]  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- UNNEST  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- UNNEST  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |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/ch2/ch2_q15.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q15.plan
index 8da830c..78616bd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q15.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q15.plan
@@ -1,129 +1,110 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$220(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$220(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- HYBRID_HASH_JOIN [$#1][$$220]  |PARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- HYBRID_HASH_JOIN [$$221][$$258]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$221]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- SORT_GROUP_BY[$$270]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$230(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$230(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- HYBRID_HASH_JOIN [$#1][$$230]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$231][$$268]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$231]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$280]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- HASH_PARTITION_EXCHANGE [$$280]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$225]  |PARTITIONED|
                                             {
                                               -- AGGREGATE  |LOCAL|
                                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                                             }
-                                      -- HASH_PARTITION_EXCHANGE [$$270]  |PARTITIONED|
-                                        -- SORT_GROUP_BY[$$215]  |PARTITIONED|
-                                                {
-                                                  -- AGGREGATE  |LOCAL|
-                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                }
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- HYBRID_HASH_JOIN [$$219, $$218][$$252, $$253]  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- REPLICATE  |PARTITIONED|
-                                                                -- HASH_PARTITION_EXCHANGE [$$241, $$240]  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ASSIGN  |PARTITIONED|
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                      -- HASH_PARTITION_EXCHANGE [$$252, $$253]  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- STREAM_SELECT  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- REPLICATE  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        -- ASSIGN  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- UNNEST  |PARTITIONED|
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$258]  |PARTITIONED|
-                              -- STREAM_PROJECT  |UNPARTITIONED|
-                                -- ASSIGN  |UNPARTITIONED|
-                                  -- AGGREGATE  |UNPARTITIONED|
-                                    -- AGGREGATE  |UNPARTITIONED|
-                                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                        -- AGGREGATE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- SORT_GROUP_BY[$$273]  |PARTITIONED|
-                                                        {
-                                                          -- AGGREGATE  |LOCAL|
-                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                        }
-                                                  -- HASH_PARTITION_EXCHANGE [$$273]  |PARTITIONED|
-                                                    -- SORT_GROUP_BY[$$239]  |PARTITIONED|
-                                                            {
-                                                              -- AGGREGATE  |LOCAL|
-                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                            }
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- HYBRID_HASH_JOIN [$$229, $$228][$$262, $$263]  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$251, $$250]  |PARTITIONED|
                                                           -- ASSIGN  |PARTITIONED|
                                                             -- STREAM_PROJECT  |PARTITIONED|
                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- HYBRID_HASH_JOIN [$$241, $$240][$$256, $$257]  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- REPLICATE  |PARTITIONED|
-                                                                      -- HASH_PARTITION_EXCHANGE [$$241, $$240]  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
-                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                  -- HASH_PARTITION_EXCHANGE [$$256, $$257]  |PARTITIONED|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$262, $$263]  |PARTITIONED|
+                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- REPLICATE  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- UNNEST  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- REPLICATE  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$268]  |PARTITIONED|
+                            -- ASSIGN  |UNPARTITIONED|
+                              -- AGGREGATE  |UNPARTITIONED|
+                                -- AGGREGATE  |UNPARTITIONED|
+                                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                    -- AGGREGATE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- SORT_GROUP_BY[$$283]  |PARTITIONED|
+                                                  {
+                                                    -- AGGREGATE  |LOCAL|
+                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                  }
+                                            -- HASH_PARTITION_EXCHANGE [$$283]  |PARTITIONED|
+                                              -- SORT_GROUP_BY[$$249]  |PARTITIONED|
+                                                      {
+                                                        -- AGGREGATE  |LOCAL|
+                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                      }
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- HYBRID_HASH_JOIN [$$251, $$250][$$266, $$267]  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- REPLICATE  |PARTITIONED|
+                                                              -- HASH_PARTITION_EXCHANGE [$$251, $$250]  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- HASH_PARTITION_EXCHANGE [$$266, $$267]  |PARTITIONED|
+                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- REPLICATE  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- UNNEST  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                    -- UNNEST  |PARTITIONED|
-                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                        -- ASSIGN  |PARTITIONED|
-                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                              -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$220]  |PARTITIONED|
+                                                                              -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$230]  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q16.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q16.plan
index 6abcb63..e2aa179 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q16.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q16.plan
@@ -1,65 +1,59 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$131(DESC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$131(DESC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- PRE_CLUSTERED_GROUP_BY[$$121, $$122, $$123]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                            -- MICRO_STABLE_SORT [$$142(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|
-                      -- 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|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$125(ASC), $$126(ASC)]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$125, $$126]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- NESTED_LOOP  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$139(DESC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$139(DESC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- PRE_CLUSTERED_GROUP_BY[$$129, $$130, $$131]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                          -- MICRO_STABLE_SORT [$$150(ASC)]  |LOCAL|
+                            -- ASSIGN  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$129(ASC), $$130(ASC), $$131(ASC)]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$129, $$130, $$131]  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_CLUSTERED_GROUP_BY[$$133, $$134]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- 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 [$$138][$$137]  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$138]  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
+                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
                                                   -- STREAM_PROJECT  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- HYBRID_HASH_JOIN [$$130][$$129]  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$129]  |PARTITIONED|
-                                                          -- STREAM_SELECT  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q17.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q17.plan
index 8896e10..6c7c721 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q17.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q17.plan
@@ -1,69 +1,59 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
               -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$i_id][$$152]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$i_id][$$143]  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$163]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_CLUSTERED_GROUP_BY[$$154]  |PARTITIONED|
+                          -- SORT_GROUP_BY[$$146]  |PARTITIONED|
                                   {
                                     -- AGGREGATE  |LOCAL|
                                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                                   }
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- SORT_GROUP_BY[$$137]  |PARTITIONED|
-                                      {
-                                        -- AGGREGATE  |LOCAL|
-                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                      }
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- HYBRID_HASH_JOIN [$$137][$$147]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- STREAM_SELECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$147]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- REPLICATE  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- UNNEST  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$143]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$146][$$156]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$146]  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$156]  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- REPLICATE  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- UNNEST  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$152]  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- UNNEST  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- UNNEST  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (test.orders)  |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/ch2/ch2_q18.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q18.plan
index ad40849..18d34b2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q18.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q18.plan
@@ -1,50 +1,44 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     -- STREAM_LIMIT  |UNPARTITIONED|
-      -- STREAM_PROJECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- SORT_MERGE_EXCHANGE [$$182(DESC), $$o_entry_d(ASC) ]  |PARTITIONED|
-              -- STREAM_LIMIT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [topK: 100] [$$182(DESC), $$o_entry_d(ASC)]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$193(DESC), $$o_entry_d(ASC) ]  |PARTITIONED|
+            -- STREAM_LIMIT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [topK: 100] [$$193(DESC), $$o_entry_d(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- SORT_GROUP_BY[$$188, $$189, $$190, $$191, $$192, $$193, $$194]  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- SORT_GROUP_BY[$$199, $$200, $$201, $$202, $$203, $$204, $$205]  |PARTITIONED|
+                                  {
+                                    -- AGGREGATE  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                  }
+                            -- HASH_PARTITION_EXCHANGE [$$199, $$200, $$201, $$202, $$203, $$204, $$205]  |PARTITIONED|
+                              -- SORT_GROUP_BY[$$175, $$183, $$184, $$185, $$179, $$180, $$181]  |PARTITIONED|
                                       {
                                         -- AGGREGATE  |LOCAL|
                                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                                       }
-                                -- HASH_PARTITION_EXCHANGE [$$188, $$189, $$190, $$191, $$192, $$193, $$194]  |PARTITIONED|
-                                  -- SORT_GROUP_BY[$$164, $$172, $$173, $$174, $$168, $$169, $$170]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- HYBRID_HASH_JOIN [$$177, $$172, $$173][$$174, $$178, $$179]  |PARTITIONED|
-                                            -- HASH_PARTITION_EXCHANGE [$$177, $$172, $$173]  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- UNNEST  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                            -- HASH_PARTITION_EXCHANGE [$$174, $$178, $$179]  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HYBRID_HASH_JOIN [$$188, $$183, $$184][$$185, $$189, $$190]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$188, $$183, $$184]  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- UNNEST  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$185, $$189, $$190]  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q19.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q19.plan
index 4030e67..75be180 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q19.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q19.plan
@@ -1,34 +1,28 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
               -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$122][$$123]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- UNNEST  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$123]  |PARTITIONED|
-                          -- STREAM_SELECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$126][$$127]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$126]  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- UNNEST  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$127]  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q2.plan
index 84f5c1d..1dd9319 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q2.plan
@@ -1,162 +1,147 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     -- STREAM_LIMIT  |UNPARTITIONED|
-      -- STREAM_PROJECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$$260(ASC), $$261(ASC), $$226(ASC) ]  |PARTITIONED|
-            -- STREAM_LIMIT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [topK: 100] [$$260(ASC), $$261(ASC), $$226(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- HYBRID_HASH_JOIN [$$238][$$239]  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$238]  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- HYBRID_HASH_JOIN [$$245][$$246]  |PARTITIONED|
-                                  -- HASH_PARTITION_EXCHANGE [$$245]  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- HYBRID_HASH_JOIN [$$251][$$228]  |PARTITIONED|
-                                          -- HASH_PARTITION_EXCHANGE [$$251]  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$280(ASC), $$281(ASC), $$247(ASC) ]  |PARTITIONED|
+          -- STREAM_LIMIT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [topK: 100] [$$280(ASC), $$281(ASC), $$247(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$260][$$261]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$260]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$267][$$268]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$267]  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- HYBRID_HASH_JOIN [$$273][$$249]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$273]  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
                                             -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- HYBRID_HASH_JOIN [$$226, $$240][$$227, $$249]  |PARTITIONED|
-                                                      -- HASH_PARTITION_EXCHANGE [$$226, $$240]  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- HYBRID_HASH_JOIN [$$247, $$262][$$248, $$271]  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$247, $$262]  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- HYBRID_HASH_JOIN [$$s_i_id][$$247]  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- HYBRID_HASH_JOIN [$$s_i_id][$$226]  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- SORT_GROUP_BY[$$271]  |PARTITIONED|
+                                                            -- SORT_GROUP_BY[$$291]  |PARTITIONED|
+                                                                    {
+                                                                      -- AGGREGATE  |LOCAL|
+                                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                    }
+                                                              -- HASH_PARTITION_EXCHANGE [$$291]  |PARTITIONED|
+                                                                -- SORT_GROUP_BY[$$245]  |PARTITIONED|
                                                                         {
                                                                           -- AGGREGATE  |LOCAL|
                                                                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                                         }
-                                                                  -- HASH_PARTITION_EXCHANGE [$$271]  |PARTITIONED|
-                                                                    -- SORT_GROUP_BY[$$225]  |PARTITIONED|
-                                                                            {
-                                                                              -- AGGREGATE  |LOCAL|
-                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                            }
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- STREAM_PROJECT  |PARTITIONED|
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- HYBRID_HASH_JOIN [$$247][$$256]  |PARTITIONED|
-                                                                              -- HASH_PARTITION_EXCHANGE [$$247]  |PARTITIONED|
-                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                        -- HYBRID_HASH_JOIN [$$269][$$277]  |PARTITIONED|
+                                                                          -- HASH_PARTITION_EXCHANGE [$$269]  |PARTITIONED|
+                                                                            -- ASSIGN  |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 (test.stock)  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                          -- HASH_PARTITION_EXCHANGE [$$277]  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- HYBRID_HASH_JOIN [$$265][$$276]  |PARTITIONED|
+                                                                                  -- HASH_PARTITION_EXCHANGE [$$265]  |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 (test.stock)  |PARTITIONED|
+                                                                                                -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
                                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                              -- HASH_PARTITION_EXCHANGE [$$256]  |PARTITIONED|
-                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    -- HYBRID_HASH_JOIN [$$243][$$255]  |PARTITIONED|
-                                                                                      -- HASH_PARTITION_EXCHANGE [$$243]  |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 (test.supplier)  |PARTITIONED|
-                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                      -- HASH_PARTITION_EXCHANGE [$$255]  |PARTITIONED|
-                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            -- HYBRID_HASH_JOIN [$$241][$$242]  |PARTITIONED|
-                                                                                              -- HASH_PARTITION_EXCHANGE [$$241]  |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 (test.nation)  |PARTITIONED|
-                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                  -- HASH_PARTITION_EXCHANGE [$$276]  |PARTITIONED|
+                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- HYBRID_HASH_JOIN [$$263][$$264]  |PARTITIONED|
+                                                                                          -- HASH_PARTITION_EXCHANGE [$$263]  |PARTITIONED|
+                                                                                            -- ASSIGN  |PARTITIONED|
                                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                 -- REPLICATE  |PARTITIONED|
-                                                                                                  -- HASH_PARTITION_EXCHANGE [$$242]  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                     -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                                                                        -- ASSIGN  |PARTITIONED|
-                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                              -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
-                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                              -- HASH_PARTITION_EXCHANGE [$$226]  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- STREAM_SELECT  |PARTITIONED|
-                                                                    -- ASSIGN  |PARTITIONED|
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                      -- HASH_PARTITION_EXCHANGE [$$227, $$249]  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                              -- HASH_PARTITION_EXCHANGE [$$264]  |PARTITIONED|
+                                                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- HASH_PARTITION_EXCHANGE [$$247]  |PARTITIONED|
+                                                            -- STREAM_SELECT  |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 (test.stock)  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                          -- HASH_PARTITION_EXCHANGE [$$228]  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- REPLICATE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$248, $$271]  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- REPLICATE  |PARTITIONED|
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                  -- HASH_PARTITION_EXCHANGE [$$246]  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$249]  |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 (test.nation)  |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 [$$242]  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- STREAM_SELECT  |PARTITIONED|
-                                          -- ASSIGN  |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 (test.supplier)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$268]  |PARTITIONED|
+                                  -- ASSIGN  |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 (test.region)  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$264]  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q20.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q20.plan
index 3b04b56..e451167 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q20.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q20.plan
@@ -1,112 +1,99 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$178(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$178(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- HYBRID_HASH_JOIN [$$172][$$173]  |PARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$172]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$191(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$191(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- HYBRID_HASH_JOIN [$$186][$$187]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$186]  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_CLUSTERED_GROUP_BY[$$177]  |PARTITIONED|
+                                  {
+                                    -- AGGREGATE  |LOCAL|
+                                      -- STREAM_SELECT  |LOCAL|
+                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                  }
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_CLUSTERED_GROUP_BY[$$164]  |PARTITIONED|
-                                      {
-                                        -- AGGREGATE  |LOCAL|
-                                          -- STREAM_SELECT  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                      }
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$164(ASC)]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$164]  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- HYBRID_HASH_JOIN [$$171][$$143]  |PARTITIONED|
-                                            -- HASH_PARTITION_EXCHANGE [$$171]  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                              -- STABLE_SORT [$$177(ASC)]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$177]  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- HYBRID_HASH_JOIN [$$185][$$156]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$185]  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$156]  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- SORT_GROUP_BY[$$200, $$201, $$202]  |PARTITIONED|
+                                                        {
+                                                          -- AGGREGATE  |LOCAL|
+                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                        }
+                                                  -- HASH_PARTITION_EXCHANGE [$$200, $$201, $$202]  |PARTITIONED|
+                                                    -- SORT_GROUP_BY[$$184, $$173, $$174]  |PARTITIONED|
+                                                            {
+                                                              -- AGGREGATE  |LOCAL|
+                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                            }
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                            -- HASH_PARTITION_EXCHANGE [$$143]  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- SORT_GROUP_BY[$$187, $$188, $$189]  |PARTITIONED|
-                                                                {
-                                                                  -- AGGREGATE  |LOCAL|
-                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                }
-                                                          -- HASH_PARTITION_EXCHANGE [$$187, $$188, $$189]  |PARTITIONED|
-                                                            -- SORT_GROUP_BY[$$170, $$160, $$161]  |PARTITIONED|
-                                                                    {
-                                                                      -- AGGREGATE  |LOCAL|
-                                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                    }
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- HYBRID_HASH_JOIN [$$170][$$174]  |PARTITIONED|
-                                                                      -- HASH_PARTITION_EXCHANGE [$$170]  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- STREAM_SELECT  |PARTITIONED|
-                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- PRE_CLUSTERED_GROUP_BY[$$166]  |PARTITIONED|
-                                                                                        {
-                                                                                          -- AGGREGATE  |LOCAL|
-                                                                                            -- STREAM_SELECT  |LOCAL|
-                                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                        }
-                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    -- STABLE_SORT [$$166(ASC)]  |PARTITIONED|
-                                                                                      -- HASH_PARTITION_EXCHANGE [$$166]  |PARTITIONED|
-                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            -- HYBRID_HASH_JOIN [$$170][$$117]  |PARTITIONED|
-                                                                                              -- HASH_PARTITION_EXCHANGE [$$170]  |PARTITIONED|
-                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                      -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
-                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                              -- HASH_PARTITION_EXCHANGE [$$117]  |PARTITIONED|
-                                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                            -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
-                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                      -- HASH_PARTITION_EXCHANGE [$$174]  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- STREAM_SELECT  |PARTITIONED|
-                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                  -- UNNEST  |PARTITIONED|
-                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- HYBRID_HASH_JOIN [$$184][$$188]  |PARTITIONED|
+                                                              -- HASH_PARTITION_EXCHANGE [$$184]  |PARTITIONED|
+                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- PRE_CLUSTERED_GROUP_BY[$$179]  |PARTITIONED|
+                                                                              {
+                                                                                -- AGGREGATE  |LOCAL|
+                                                                                  -- STREAM_SELECT  |LOCAL|
+                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                              }
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- STABLE_SORT [$$179(ASC)]  |PARTITIONED|
+                                                                            -- HASH_PARTITION_EXCHANGE [$$179]  |PARTITIONED|
+                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- HYBRID_HASH_JOIN [$$184][$$130]  |PARTITIONED|
+                                                                                    -- HASH_PARTITION_EXCHANGE [$$184]  |PARTITIONED|
                                                                                       -- ASSIGN  |PARTITIONED|
-                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                    -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                            -- STREAM_PROJECT  |PARTITIONED|
                                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$173]  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                              -- HASH_PARTITION_EXCHANGE [$$188]  |PARTITIONED|
+                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- UNNEST  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$187]  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q21.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q21.plan
index 6c845e0..3ed88a3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q21.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q21.plan
@@ -3,114 +3,100 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- SORT_MERGE_EXCHANGE [$$su_name(ASC) ]  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_LIMIT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- SORT_GROUP_BY[$$377]  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- HASH_PARTITION_EXCHANGE [$$377]  |PARTITIONED|
-                      -- SORT_GROUP_BY[$$su_name]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_LIMIT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- SORT_GROUP_BY[$$399]  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- HASH_PARTITION_EXCHANGE [$$399]  |PARTITIONED|
+                    -- SORT_GROUP_BY[$$su_name]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- SORT_GROUP_BY[$$368, $$369, $$370, $$371, $$372, $$373, $$374, $$375]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- SORT_GROUP_BY[$$390, $$391, $$392, $$393, $$394, $$395, $$396, $$397]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- HASH_PARTITION_EXCHANGE [$$390, $$391, $$392, $$393, $$394, $$395, $$396, $$397]  |PARTITIONED|
+                                  -- SORT_GROUP_BY[$$352, $$378, $$377, $$354, $$355, $$351, $$350, $$379]  |PARTITIONED|
                                           {
                                             -- AGGREGATE  |LOCAL|
                                               -- NESTED_TUPLE_SOURCE  |LOCAL|
                                           }
-                                    -- HASH_PARTITION_EXCHANGE [$$368, $$369, $$370, $$371, $$372, $$373, $$374, $$375]  |PARTITIONED|
-                                      -- SORT_GROUP_BY[$$330, $$356, $$355, $$332, $$333, $$329, $$328, $$357]  |PARTITIONED|
-                                              {
-                                                -- AGGREGATE  |LOCAL|
-                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                              }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                          -- NESTED_LOOP  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- NESTED_LOOP  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- HYBRID_HASH_JOIN [$$329, $$328][$$330, $$340]  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$329, $$328]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- HYBRID_HASH_JOIN [$$333][$$344]  |PARTITIONED|
-                                                                -- HASH_PARTITION_EXCHANGE [$$333]  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- HYBRID_HASH_JOIN [$$332][$$343]  |PARTITIONED|
-                                                                        -- HASH_PARTITION_EXCHANGE [$$332]  |PARTITIONED|
+                                                  -- HYBRID_HASH_JOIN [$$351, $$350][$$352, $$362]  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$351, $$350]  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- HYBRID_HASH_JOIN [$$355][$$366]  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$355]  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- HYBRID_HASH_JOIN [$$354][$$365]  |PARTITIONED|
+                                                                    -- HASH_PARTITION_EXCHANGE [$$354]  |PARTITIONED|
+                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
                                                                           -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- STREAM_SELECT  |PARTITIONED|
-                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
-                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                        -- HASH_PARTITION_EXCHANGE [$$343]  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                -- HASH_PARTITION_EXCHANGE [$$344]  |PARTITIONED|
-                                                                  -- ASSIGN  |PARTITIONED|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                    -- HASH_PARTITION_EXCHANGE [$$365]  |PARTITIONED|
                                                                       -- ASSIGN  |PARTITIONED|
                                                                         -- STREAM_PROJECT  |PARTITIONED|
                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
+                                                                            -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$330, $$340]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- STREAM_SELECT  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$366]  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
                                                                 -- ASSIGN  |PARTITIONED|
                                                                   -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- UNNEST  |PARTITIONED|
-                                                                      -- STREAM_SELECT  |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 (test.orders)  |PARTITIONED|
-                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- UNNEST  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- STREAM_SELECT  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ASSIGN  |PARTITIONED|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                      -- ASSIGN  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- REPLICATE  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$352, $$362]  |PARTITIONED|
+                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- UNNEST  |PARTITIONED|
+                                                            -- STREAM_SELECT  |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 (test.orders)  |PARTITIONED|
                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- UNNEST  |PARTITIONED|
+                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                      -- ASSIGN  |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 (test.orders)  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q22.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q22.plan
index 19a82f7..baf25f4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q22.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q22.plan
@@ -1,110 +1,100 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$#1(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$168]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$168]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$136]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- SORT_GROUP_BY[$$165]  |PARTITIONED|
-                                      {
-                                        -- AGGREGATE  |LOCAL|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$#1(ASC) ]  |PARTITIONED|
+        -- SORT_GROUP_BY[$$174]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$174]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$142]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$171]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- HASH_PARTITION_EXCHANGE [$$171]  |PARTITIONED|
+                            -- PRE_CLUSTERED_GROUP_BY[$$166]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
                                           -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                      }
-                                -- HASH_PARTITION_EXCHANGE [$$165]  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$160]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$160(ASC)]  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$149, $$151, $$153][$$148, $$150, $$152]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$149, $$151, $$153]  |PARTITIONED|
-                                                  -- RUNNING_AGGREGATE  |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 [$$79][$$141]  |PARTITIONED|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$166(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$155, $$157, $$159][$$154, $$156, $$158]  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$155, $$157, $$159]  |PARTITIONED|
+                                            -- RUNNING_AGGREGATE  |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 [$$85][$$147]  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ASSIGN  |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|
-                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- DATASOURCE_SCAN (test.customer)  |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|
-                                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                              -- UNNEST  |UNPARTITIONED|
-                                                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |UNPARTITIONED|
-                                                              -- ASSIGN  |UNPARTITIONED|
-                                                                -- AGGREGATE  |UNPARTITIONED|
-                                                                  -- AGGREGATE  |UNPARTITIONED|
-                                                                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                                      -- AGGREGATE  |PARTITIONED|
+                                                                    -- REPLICATE  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                         -- STREAM_PROJECT  |PARTITIONED|
                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- HYBRID_HASH_JOIN [$$87][$$142]  |PARTITIONED|
+                                                                            -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                    -- STREAM_SELECT  |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 (test.customer)  |PARTITIONED|
-                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- REPLICATE  |PARTITIONED|
+                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                      -- UNNEST  |UNPARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                      -- ASSIGN  |UNPARTITIONED|
+                                                        -- AGGREGATE  |UNPARTITIONED|
+                                                          -- AGGREGATE  |UNPARTITIONED|
+                                                            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                              -- AGGREGATE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- HYBRID_HASH_JOIN [$$93][$$148]  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                 -- REPLICATE  |PARTITIONED|
-                                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                    -- UNNEST  |UNPARTITIONED|
-                                                                                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$148, $$150, $$152]  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- REPLICATE  |PARTITIONED|
+                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                            -- UNNEST  |UNPARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$154, $$156, $$158]  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |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/ch2/ch2_q3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q3.plan
index a2ffaea..ce1376c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q3.plan
@@ -1,56 +1,50 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$204(DESC), $$o_entry_d(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$204(DESC), $$o_entry_d(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- SORT_GROUP_BY[$$211, $$212, $$213, $$214]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- HASH_PARTITION_EXCHANGE [$$211, $$212, $$213, $$214]  |PARTITIONED|
-                  -- SORT_GROUP_BY[$$206, $$188, $$189, $$190]  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- HYBRID_HASH_JOIN [$$188, $$189, $$206][$$201, $$202, $$203]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$188, $$189, $$206]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$220(DESC), $$o_entry_d(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$220(DESC), $$o_entry_d(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- SORT_GROUP_BY[$$226, $$227, $$228, $$229]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- HASH_PARTITION_EXCHANGE [$$226, $$227, $$228, $$229]  |PARTITIONED|
+                -- SORT_GROUP_BY[$$221, $$204, $$205, $$206]  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$204, $$205, $$221][$$217, $$218, $$219]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$204, $$205, $$221]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- UNNEST  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$214, $$204, $$205][$$213, $$215, $$216]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$214, $$204, $$205]  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$213, $$215, $$216]  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$217, $$218, $$219]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- UNNEST  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- HYBRID_HASH_JOIN [$$198, $$188, $$189][$$197, $$199, $$200]  |PARTITIONED|
-                                            -- HASH_PARTITION_EXCHANGE [$$198, $$188, $$189]  |PARTITIONED|
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                            -- HASH_PARTITION_EXCHANGE [$$197, $$199, $$200]  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- STREAM_SELECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$201, $$202, $$203]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.neworder)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.neworder)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q4.plan
index 5818a43..f4c7d0d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q4.plan
@@ -1,36 +1,33 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$o_ol_cnt(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$77]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$77]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$68]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$o_ol_cnt(ASC) ]  |PARTITIONED|
+        -- SORT_GROUP_BY[$$82]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$82]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$73]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- SUBPLAN  |PARTITIONED|
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- STREAM_SELECT  |LOCAL|
-                                      -- ASSIGN  |LOCAL|
-                                        -- UNNEST  |LOCAL|
-                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
-                          -- STREAM_SELECT  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- SUBPLAN  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- STREAM_SELECT  |LOCAL|
+                                  -- ASSIGN  |LOCAL|
+                                    -- UNNEST  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |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/ch2/ch2_q5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q5.plan
index 30ae23f..bf45fd7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q5.plan
@@ -1,98 +1,86 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$#2(DESC) ]  |PARTITIONED|
-          -- STABLE_SORT [$#2(DESC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- SORT_GROUP_BY[$$277]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$#2(DESC) ]  |PARTITIONED|
+        -- STABLE_SORT [$#2(DESC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- SORT_GROUP_BY[$$291]  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- HASH_PARTITION_EXCHANGE [$$291]  |PARTITIONED|
+                    -- SORT_GROUP_BY[$$275]  |PARTITIONED|
                             {
                               -- AGGREGATE  |LOCAL|
                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                             }
-                      -- HASH_PARTITION_EXCHANGE [$$277]  |PARTITIONED|
-                        -- SORT_GROUP_BY[$$261]  |PARTITIONED|
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- HYBRID_HASH_JOIN [$$260, $$254][$$252, $$255]  |PARTITIONED|
-                                  -- HASH_PARTITION_EXCHANGE [$$260, $$254]  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- HYBRID_HASH_JOIN [$$231, $$232][$$240, $$267]  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$231, $$232]  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$240, $$267]  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$274, $$268][$$266, $$269]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$274, $$268]  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- HYBRID_HASH_JOIN [$$245, $$246][$$254, $$281]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$245, $$246]  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- HYBRID_HASH_JOIN [$$246, $$240, $$249][$$262, $$263, $$264]  |PARTITIONED|
-                                                      -- HASH_PARTITION_EXCHANGE [$$246, $$240, $$249]  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$254, $$281]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- HYBRID_HASH_JOIN [$$260, $$254, $$263][$$276, $$277, $$278]  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$260, $$254, $$263]  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- UNNEST  |PARTITIONED|
+                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$276, $$277, $$278]  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- HYBRID_HASH_JOIN [$$258][$$274]  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$258]  |PARTITIONED|
                                                           -- ASSIGN  |PARTITIONED|
                                                             -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- UNNEST  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- STREAM_SELECT  |PARTITIONED|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                      -- ASSIGN  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                      -- HASH_PARTITION_EXCHANGE [$$262, $$263, $$264]  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- HYBRID_HASH_JOIN [$$244][$$260]  |PARTITIONED|
-                                                              -- HASH_PARTITION_EXCHANGE [$$244]  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$274]  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- HYBRID_HASH_JOIN [$$256][$$257]  |PARTITIONED|
+                                                                -- HASH_PARTITION_EXCHANGE [$$256]  |PARTITIONED|
                                                                   -- ASSIGN  |PARTITIONED|
                                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
+                                                                        -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                              -- HASH_PARTITION_EXCHANGE [$$260]  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- HYBRID_HASH_JOIN [$$242][$$243]  |PARTITIONED|
-                                                                      -- HASH_PARTITION_EXCHANGE [$$242]  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
-                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                      -- HASH_PARTITION_EXCHANGE [$$243]  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- STREAM_SELECT  |PARTITIONED|
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                  -- HASH_PARTITION_EXCHANGE [$$252, $$255]  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                -- HASH_PARTITION_EXCHANGE [$$257]  |PARTITIONED|
+                                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$266, $$269]  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q6.plan
index 3b05b97..e8e1b49 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q6.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q6.plan
@@ -1,20 +1,15 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- UNNEST  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- UNNEST  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |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/ch2/ch2_q7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q7.plan
index 4572727..5949891 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q7.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q7.plan
@@ -1,102 +1,89 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$su_nationkey(ASC), $#1(ASC), $#2(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$310, $$311, $$312]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$310, $$311, $$312]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$262, $$258, $$259]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$su_nationkey(ASC), $#1(ASC), $#2(ASC) ]  |PARTITIONED|
+        -- SORT_GROUP_BY[$$325, $$326, $$327]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$325, $$326, $$327]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$277, $$273, $$274]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- HYBRID_HASH_JOIN [$$288, $$284][$$262, $$285]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$288, $$284]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- HYBRID_HASH_JOIN [$$260, $$261][$$295, $$296]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$260, $$261]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$295, $$296]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$303, $$299][$$277, $$300]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$303, $$299]  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$275, $$276][$$310, $$311]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$275, $$276]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$276, $$278, $$280][$$290, $$291, $$292]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$276, $$278, $$280]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- UNNEST  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$310, $$311]  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$291, $$293, $$295][$$305, $$306, $$307]  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$291, $$293, $$295]  |PARTITIONED|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- UNNEST  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$305, $$306, $$307]  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- HYBRID_HASH_JOIN [$$288][$$304]  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$288]  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$304]  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- NESTED_LOOP  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- REPLICATE  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                            -- REPLICATE  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                 -- ASSIGN  |PARTITIONED|
                                                                   -- STREAM_PROJECT  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                                                      -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$290, $$291, $$292]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- HYBRID_HASH_JOIN [$$273][$$289]  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$273]  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$289]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- NESTED_LOOP  |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|
-                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
-                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                  -- REPLICATE  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        -- ASSIGN  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$262, $$285]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$277, $$300]  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q8.plan
index 3c62aa7..0087b4b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q8.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q8.plan
@@ -1,128 +1,114 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$#1(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$333]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$333]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$278]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$#1(ASC) ]  |PARTITIONED|
+        -- SORT_GROUP_BY[$$349]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$349]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$294]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- HYBRID_HASH_JOIN [$$304][$$325]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$304]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- HYBRID_HASH_JOIN [$$280, $$279][$$290, $$320]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$280, $$279]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$290, $$320]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$320][$$341]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$320]  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$296, $$295][$$306, $$336]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$296, $$295]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$297, $$299, $$301][$$317, $$318, $$316]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$297, $$299, $$301]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- HYBRID_HASH_JOIN [$$290][$$308]  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$290]  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$306, $$336]  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$313, $$315, $$317][$$333, $$334, $$332]  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$313, $$315, $$317]  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- HYBRID_HASH_JOIN [$$306][$$324]  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$306]  |PARTITIONED|
+                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- UNNEST  |PARTITIONED|
                                                           -- STREAM_SELECT  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- UNNEST  |PARTITIONED|
-                                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        -- ASSIGN  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$308]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$324]  |PARTITIONED|
+                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$333, $$334, $$332]  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- HYBRID_HASH_JOIN [$$311][$$331]  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$311]  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$331]  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- HYBRID_HASH_JOIN [$$308][$$309]  |PARTITIONED|
+                                                          -- HASH_PARTITION_EXCHANGE [$$308]  |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 (test.nation)  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- HASH_PARTITION_EXCHANGE [$$309]  |PARTITIONED|
                                                             -- STREAM_SELECT  |PARTITIONED|
                                                               -- ASSIGN  |PARTITIONED|
                                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
+                                                                    -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$317, $$318, $$316]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- HYBRID_HASH_JOIN [$$295][$$315]  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$295]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$315]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- HYBRID_HASH_JOIN [$$292][$$293]  |PARTITIONED|
-                                                                -- HASH_PARTITION_EXCHANGE [$$292]  |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 (test.nation)  |PARTITIONED|
-                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                -- HASH_PARTITION_EXCHANGE [$$293]  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                                      -- ASSIGN  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$325]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$309][$$310]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$309]  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$310]  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$341]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$325][$$326]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$325]  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$326]  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- REPLICATE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q9.plan
index 97037d6..5fd9860 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q9.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ch2/ch2_q9.plan
@@ -1,81 +1,71 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$n_name(ASC), $#1(DESC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$217, $$218]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$217, $$218]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$213, $$186]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$n_name(ASC), $#1(DESC) ]  |PARTITIONED|
+        -- SORT_GROUP_BY[$$229, $$230]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$229, $$230]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$225, $$198]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- HYBRID_HASH_JOIN [$$198][$$212]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$198]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- HYBRID_HASH_JOIN [$$188, $$187][$$194, $$205]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$188]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$210][$$224]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$210]  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$200, $$199][$$206, $$217]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$200]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$194][$$195]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$194]  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$206][$$207]  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$206]  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- UNNEST  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
                                                   -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- UNNEST  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$195]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$212]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$201][$$202]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$201]  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$202]  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$207]  |PARTITIONED|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$224]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$213][$$214]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$213]  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$214]  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |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 6c4b299..8617f74 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/collocated.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/collocated.plan
@@ -1,21 +1,18 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$32][$$33]  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$32]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- HYBRID_HASH_JOIN [$$32][$$33]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$32]  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (colocated.Users)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (colocated.Users)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$33]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$33]  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (colocated.Visitors)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (colocated.Visitors)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/column-pushdown/meta.001.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/column-pushdown/meta.001.plan
index 01a8496..d53d82a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/column-pushdown/meta.001.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/column-pushdown/meta.001.plan
@@ -1,62 +1,100 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$194(DESC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$194(DESC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- NESTED_LOOP  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- SORT_GROUP_BY[$$223]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- HASH_PARTITION_EXCHANGE [$$223]  |PARTITIONED|
-                          -- SORT_GROUP_BY[$$178]  |PARTITIONED|
-                                  {
-                                    -- AGGREGATE  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                  }
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$192][$$191]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$192]  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- HYBRID_HASH_JOIN [$$207][$$206]  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- REPLICATE  |PARTITIONED|
-                                                      -- HASH_PARTITION_EXCHANGE [$$212]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$194(DESC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$194(DESC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- SORT_GROUP_BY[$$223]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- HASH_PARTITION_EXCHANGE [$$223]  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$178]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$192][$$191]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$192]  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$207][$$206]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- REPLICATE  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$212]  |PARTITIONED|
+                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
                                                         -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- STREAM_SELECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (test.nation)  |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 [$$211]  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$191]  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- REPLICATE  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$211]  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$191]  |PARTITIONED|
+                                    -- ASSIGN  |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 (test.stock)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- ASSIGN  |UNPARTITIONED|
+                      -- AGGREGATE  |UNPARTITIONED|
+                        -- ASSIGN  |UNPARTITIONED|
+                          -- AGGREGATE  |UNPARTITIONED|
+                            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                              -- AGGREGATE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$210][$$209]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$210]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$212][$$211]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- REPLICATE  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$212]  |PARTITIONED|
+                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- REPLICATE  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$211]  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$209]  |PARTITIONED|
                                         -- ASSIGN  |PARTITIONED|
                                           -- ASSIGN  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -67,53 +105,3 @@
                                                       -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |UNPARTITIONED|
-                        -- ASSIGN  |UNPARTITIONED|
-                          -- AGGREGATE  |UNPARTITIONED|
-                            -- STREAM_PROJECT  |UNPARTITIONED|
-                              -- ASSIGN  |UNPARTITIONED|
-                                -- AGGREGATE  |UNPARTITIONED|
-                                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                    -- AGGREGATE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- HYBRID_HASH_JOIN [$$210][$$209]  |PARTITIONED|
-                                            -- HASH_PARTITION_EXCHANGE [$$210]  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- HYBRID_HASH_JOIN [$$212][$$211]  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- REPLICATE  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$212]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- STREAM_SELECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- REPLICATE  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$211]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                            -- HASH_PARTITION_EXCHANGE [$$209]  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |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 (test.stock)  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |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 e68bf78..44453cd 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
@@ -1,68 +1,61 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- SORT_GROUP_BY[$$194, $$195]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$194, $$195]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$177, $$178]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- SORT_GROUP_BY[$$194, $$195]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$194, $$195]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$177, $$178]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- NESTED_LOOP  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- NESTED_LOOP  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
                             -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- PRE_CLUSTERED_GROUP_BY[$$181]  |PARTITIONED|
-                                                {
-                                                  -- AGGREGATE  |LOCAL|
-                                                    -- STREAM_SELECT  |LOCAL|
-                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$181]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$181(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$181]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |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|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_SELECT  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- REPLICATE  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                        -- ASSIGN  |UNPARTITIONED|
-                                                          -- UNNEST  |UNPARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- AGGREGATE  |UNPARTITIONED|
-                              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                -- AGGREGATE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
+                                            -- NESTED_LOOP  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- STREAM_SELECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |UNPARTITIONED|
+                                                  -- UNNEST  |UNPARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |UNPARTITIONED|
+                          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                            -- AGGREGATE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |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 4d5fe26..edf4154 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
@@ -1,11 +1,10 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$26(ASC), $$27(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$26(ASC), $$27(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- BTREE_SEARCH (test.Points.Points)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$26(ASC), $$27(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$26(ASC), $$27(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |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/copy-to/copy-to.1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/copy-to/copy-to.1.plan
index cc2cf3b..4e5d202 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/copy-to/copy-to.1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/copy-to/copy-to.1.plan
@@ -9,13 +9,12 @@
                 -- REPLICATE  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (test.OpenDataset)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.OpenDataset)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
               -- BROADCAST_EXCHANGE  |PARTITIONED|
                 -- AGGREGATE  |UNPARTITIONED|
                   -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
@@ -24,10 +23,9 @@
                         -- REPLICATE  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.OpenDataset)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test.OpenDataset)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/copy-to/copy-to.3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/copy-to/copy-to.3.plan
index 070698a..e94b117 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/copy-to/copy-to.3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/copy-to/copy-to.3.plan
@@ -5,10 +5,9 @@
         -- STABLE_SORT [$$32(DESC), $$33(ASC)]  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.OpenDataset)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (test.OpenDataset)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/cust_group_no_agg.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/cust_group_no_agg.plan
index 7afa33f..b2c62ae 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/cust_group_no_agg.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/cust_group_no_agg.plan
@@ -1,15 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- SORT_GROUP_BY[$$34]  |PARTITIONED|
+    -- SORT_GROUP_BY[$$37]  |PARTITIONED|
             {
               -- AGGREGATE  |LOCAL|
                 -- NESTED_TUPLE_SOURCE  |LOCAL|
             }
-      -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- DATASOURCE_SCAN (group_no_agg.Customers)  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+      -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN (group_no_agg.Customers)  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |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 f2f336d..6cab866 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
@@ -1,27 +1,26 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$99]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$99(ASC)]  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$99][$$101]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$99]  |PARTITIONED|
-                          -- DATASOURCE_SCAN (custorder.Customers)  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$99]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$99(ASC)]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |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 [$$101]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$101]  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (custorder.Orders)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (custorder.Orders)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/external-cross-product.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/external-cross-product.plan
index fd95cc1..8c77a56 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/external-cross-product.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/external-cross-product.plan
@@ -1,45 +1,42 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- NESTED_LOOP  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- NESTED_LOOP  |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 (test.Orders)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- REPLICATE  |PARTITIONED|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (test.Orders)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- NESTED_LOOP  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- ASSIGN  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- REPLICATE  |PARTITIONED|
-                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.Orders)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.Orders)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- REPLICATE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.Orders)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- REPLICATE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |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/extract-common-operators/extract-common-operators.01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/extract-common-operators/extract-common-operators.01.plan
index 677a45f..e0c2670 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/extract-common-operators/extract-common-operators.01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/extract-common-operators/extract-common-operators.01.plan
@@ -1,98 +1,92 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$381(ASC), $$382(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$381(ASC), $$382(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- HYBRID_HASH_JOIN [$$390][$$386]  |PARTITIONED|
-                    -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$381(ASC), $$382(ASC)] HASH:[$$390]  |PARTITIONED|
-                      -- STABLE_SORT [$$381(ASC), $$382(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- HYBRID_HASH_JOIN [$$390][$$384]  |PARTITIONED|
-                                -- HASH_PARTITION_EXCHANGE [$$390]  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$405(ASC), $$406(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$405(ASC), $$406(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- HYBRID_HASH_JOIN [$$414][$$410]  |PARTITIONED|
+                  -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$405(ASC), $$406(ASC)] HASH:[$$414]  |PARTITIONED|
+                    -- STABLE_SORT [$$405(ASC), $$406(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$414][$$408]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$414]  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- REPLICATE  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (Metadata.Synonym)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$408]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$408, $$407][$$ds_name, $$dv_name]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$408, $$407]  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN (Metadata.Synonym)  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (Metadata.Dataset)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- HASH_PARTITION_EXCHANGE [$$384]  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- HYBRID_HASH_JOIN [$$384, $$383][$$ds_name, $$dv_name]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$ds_name, $$dv_name]  |PARTITIONED|
+                                        -- NESTED_LOOP  |UNPARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                            -- UNNEST  |UNPARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                            -- REPLICATE  |UNPARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$410]  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$428][$$412]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$428]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$410][$$syn_name]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$410]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- REPLICATE  |PARTITIONED|
-                                            -- HASH_PARTITION_EXCHANGE [$$384, $$383]  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- DATASOURCE_SCAN (Metadata.Dataset)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$ds_name, $$dv_name]  |PARTITIONED|
-                                          -- NESTED_LOOP  |UNPARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                              -- UNNEST  |UNPARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                              -- REPLICATE  |UNPARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                  -- UNNEST  |UNPARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$386]  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- HYBRID_HASH_JOIN [$$404][$$388]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$404]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$386][$$syn_name]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$386]  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- REPLICATE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (Metadata.Synonym)  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- DATASOURCE_SCAN (Metadata.Synonym)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$syn_name]  |PARTITIONED|
-                                      -- UNNEST  |UNPARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$388]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$388, $$387][$$ds_name, $$dv_name]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$384, $$383]  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$syn_name]  |PARTITIONED|
+                                    -- UNNEST  |UNPARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$412]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$412, $$411][$$ds_name, $$dv_name]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$408, $$407]  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (Metadata.Dataset)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (Metadata.Dataset)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$ds_name, $$dv_name]  |PARTITIONED|
-                                      -- NESTED_LOOP  |UNPARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                          -- STREAM_PROJECT  |UNPARTITIONED|
-                                            -- ASSIGN  |UNPARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$ds_name, $$dv_name]  |PARTITIONED|
+                                    -- NESTED_LOOP  |UNPARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                        -- ASSIGN  |UNPARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                            -- REPLICATE  |UNPARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                -- REPLICATE  |UNPARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                    -- UNNEST  |UNPARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                          -- UNNEST  |UNPARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
\ No newline at end of file
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                        -- UNNEST  |UNPARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/inverted-btree-search-return-optional-field.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/inverted-btree-search-return-optional-field.plan
index 21e4b56..a76d642 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/inverted-btree-search-return-optional-field.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/inverted-btree-search-return-optional-field.plan
@@ -1,29 +1,27 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     -- STREAM_LIMIT  |UNPARTITIONED|
-      -- STREAM_PROJECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$$39(DESC) ]  |PARTITIONED|
-            -- STREAM_LIMIT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [topK: 2147483647] [$$39(DESC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- HYBRID_HASH_JOIN [$$26][$$41]  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- STREAM_SELECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$40(DESC) ]  |PARTITIONED|
+          -- STREAM_LIMIT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [topK: 2147483647] [$$40(DESC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$27][$$42]  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (twitter.ds_tweet.ds_tweet)  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (twitter.ds_tweet.ds_tweet)  |PARTITIONED|
+                                    -- STABLE_SORT [$$53(ASC)]  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
+                                        -- SINGLE_PARTITION_INVERTED_INDEX_SEARCH (twitter.ds_tweet.text_idx)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- SINGLE_PARTITION_INVERTED_INDEX_SEARCH (twitter.ds_tweet.text_idx)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- UNNEST  |UNPARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- UNNEST  |UNPARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_0.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_0.plan
index cb2b171..190d740 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_0.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_0.plan
@@ -1,11 +1,10 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- DATASOURCE_SCAN (KeyVerse.KVStore)  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN (KeyVerse.KVStore)  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_1.plan
index 38c95ba..190d740 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_1.plan
@@ -1,12 +1,10 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- DATASOURCE_SCAN (KeyVerse.KVStore)  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN (KeyVerse.KVStore)  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_2.plan
index 38c95ba..190d740 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_2.plan
@@ -1,12 +1,10 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- DATASOURCE_SCAN (KeyVerse.KVStore)  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN (KeyVerse.KVStore)  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_3.plan
index cb2b171..190d740 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_3.plan
@@ -1,11 +1,10 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- DATASOURCE_SCAN (KeyVerse.KVStore)  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN (KeyVerse.KVStore)  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_4.plan
index 38c95ba..190d740 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_4.plan
@@ -1,12 +1,10 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- DATASOURCE_SCAN (KeyVerse.KVStore)  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN (KeyVerse.KVStore)  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_5.plan
index 38c95ba..190d740 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_5.plan
@@ -1,12 +1,10 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- DATASOURCE_SCAN (KeyVerse.KVStore)  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN (KeyVerse.KVStore)  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_0.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_0.plan
index ece21bd..ac9bc4a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_0.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_0.plan
@@ -2,52 +2,46 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- UNION_ALL  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.OfficerLocations.OfficerLocations)  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH (test.OfficerLocations.OfficerLocations)  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+                        -- SPLIT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- SPLIT  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- RTREE_SEARCH (test.OfficerLocations.o_location)  |PARTITIONED|
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (test.KVStore)  |PARTITIONED|
-                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- RTREE_SEARCH (test.OfficerLocations.o_location)  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.KVStore)  |PARTITIONED|
+                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- SPLIT  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- SPLIT  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- RTREE_SEARCH (test.OfficerLocations.o_location)  |PARTITIONED|
-                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- STREAM_SELECT  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (test.KVStore)  |PARTITIONED|
-                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- RTREE_SEARCH (test.OfficerLocations.o_location)  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.KVStore)  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_1.plan
index aa7008a..569c903 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_1.plan
@@ -1,25 +1,22 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- NESTED_LOOP  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- NESTED_LOOP  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.KVStore)  |PARTITIONED|
-                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.OfficerLocations)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.KVStore)  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- BROADCAST_EXCHANGE  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (test.OfficerLocations)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_2.plan
index a0caf5f..cccf7a1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_2.plan
@@ -1,25 +1,22 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$35][$$36]  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.KVStore)  |PARTITIONED|
-                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.OfficerLocations)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- HYBRID_HASH_JOIN [$$37][$$38]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.KVStore)  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (test.OfficerLocations)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_3.plan
index a0caf5f..cccf7a1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_3.plan
@@ -1,25 +1,22 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$35][$$36]  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.KVStore)  |PARTITIONED|
-                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.OfficerLocations)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- HYBRID_HASH_JOIN [$$37][$$38]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.KVStore)  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (test.OfficerLocations)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/flwr/at07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/flwr/at07.plan
index a263d35..5e95e6a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/flwr/at07.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/flwr/at07.plan
@@ -1,40 +1,35 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$#2(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$#2(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- NESTED_LOOP  |PARTITIONED|
-                    -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- RUNNING_AGGREGATE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- SORT_MERGE_EXCHANGE [$$99(ASC) ]  |PARTITIONED|
-                                  -- STABLE_SORT [$$99(ASC)]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$#2(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$#2(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
+                  -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- RUNNING_AGGREGATE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- SORT_MERGE_EXCHANGE [$$108(ASC) ]  |PARTITIONED|
+                            -- STABLE_SORT [$$108(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- REPLICATE  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- DATASOURCE_SCAN (test.FacebookUsers)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |UNPARTITIONED|
-                        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                          -- AGGREGATE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- REPLICATE  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.FacebookUsers)  |PARTITIONED|
+                                      -- REPLICATE  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.FacebookUsers)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.FacebookUsers)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/gby_partitioning_property_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/gby_partitioning_property_01.plan
index 16d2e46..415ce24 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/gby_partitioning_property_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/gby_partitioning_property_01.plan
@@ -1,36 +1,33 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$70]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- SORT_GROUP_BY[$$60]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$60][$$63]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$60]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$74]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- SORT_GROUP_BY[$$64]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$64][$$67]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$64]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.GleambookUsers)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.GleambookUsers)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$63]  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$67]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.GleambookMessages)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.GleambookMessages)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/gby-case-01.3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/gby-case-01.3.plan
index 3e8d666..87ae388 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/gby-case-01.3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/gby-case-01.3.plan
@@ -1,24 +1,22 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$x(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$96]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$96]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$84]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$x(ASC) ]  |PARTITIONED|
+        -- SORT_GROUP_BY[$$99]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$99]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$87]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/gby-case-01.4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/gby-case-01.4.plan
index 9b25048..8db82ea 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/gby-case-01.4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/gby-case-01.4.plan
@@ -1,24 +1,22 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$x(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$124]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$124]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$109]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$x(ASC) ]  |PARTITIONED|
+        -- SORT_GROUP_BY[$$128]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$128]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$113]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/grouping-sets-1.1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/grouping-sets-1.1.plan
index dce7774..5771a6e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/grouping-sets-1.1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/grouping-sets-1.1.plan
@@ -12,112 +12,101 @@
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- UNION_ALL  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- SORT_GROUP_BY[$$1083]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- HASH_PARTITION_EXCHANGE [$$1083]  |PARTITIONED|
-                                      -- SORT_GROUP_BY[$$244]  |PARTITIONED|
-                                              {
-                                                -- AGGREGATE  |LOCAL|
-                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                              }
-                                        -- 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|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$1083]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- HASH_PARTITION_EXCHANGE [$$1083]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$244]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |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 (test.tenk)  |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|
+                                -- SORT_GROUP_BY[$$1085]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- HASH_PARTITION_EXCHANGE [$$1085]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$245]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |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 (test.tenk)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- SORT_GROUP_BY[$$1087, $$1088]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- HASH_PARTITION_EXCHANGE [$$1087, $$1088]  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$246, $$247]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |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 (test.tenk)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- SORT_GROUP_BY[$$1090]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- HASH_PARTITION_EXCHANGE [$$1090]  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$248]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
                               -- ASSIGN  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- SORT_GROUP_BY[$$1085]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- HASH_PARTITION_EXCHANGE [$$1085]  |PARTITIONED|
-                                      -- SORT_GROUP_BY[$$245]  |PARTITIONED|
-                                              {
-                                                -- AGGREGATE  |LOCAL|
-                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                              }
-                                        -- 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 (test.tenk)  |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|
-                              -- SORT_GROUP_BY[$$1087, $$1088]  |PARTITIONED|
-                                      {
-                                        -- AGGREGATE  |LOCAL|
-                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                      }
-                                -- HASH_PARTITION_EXCHANGE [$$1087, $$1088]  |PARTITIONED|
-                                  -- SORT_GROUP_BY[$$246, $$247]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
+                                  -- 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|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (test.tenk)  |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|
-                      -- SORT_GROUP_BY[$$1090]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- HASH_PARTITION_EXCHANGE [$$1090]  |PARTITIONED|
-                          -- SORT_GROUP_BY[$$248]  |PARTITIONED|
-                                  {
-                                    -- AGGREGATE  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                  }
-                            -- 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|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/grouping-sets-1.2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/grouping-sets-1.2.plan
index d3e260c..6d00ac3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/grouping-sets-1.2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/grouping-sets-1.2.plan
@@ -1,8 +1,8 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$1224(ASC), $$1225(ASC), $$1226(ASC), $$1227(ASC) ]  |PARTITIONED|
-        -- STABLE_SORT [$$1224(ASC), $$1225(ASC), $$1226(ASC), $$1227(ASC)]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$1231(ASC), $$1232(ASC), $$1233(ASC), $$1234(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$1231(ASC), $$1232(ASC), $$1233(ASC), $$1234(ASC)]  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- UNION_ALL  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -28,106 +28,45 @@
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                         -- UNION_ALL  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- SORT_GROUP_BY[$$10189, $$10190, $$10191, $$10192]  |PARTITIONED|
-                                                                          {
-                                                                            -- AGGREGATE  |LOCAL|
-                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                          }
-                                                                    -- HASH_PARTITION_EXCHANGE [$$10189, $$10190, $$10191, $$10192]  |PARTITIONED|
-                                                                      -- SORT_GROUP_BY[$$717, $$718, $$719, $$720]  |PARTITIONED|
-                                                                              {
-                                                                                -- AGGREGATE  |LOCAL|
-                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                              }
-                                                                        -- 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|
-                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            -- DATASOURCE_SCAN (test.tenk)  |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|
-                                                                  -- SORT_GROUP_BY[$$10194, $$10195, $$10196]  |PARTITIONED|
-                                                                          {
-                                                                            -- AGGREGATE  |LOCAL|
-                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                          }
-                                                                    -- HASH_PARTITION_EXCHANGE [$$10194, $$10195, $$10196]  |PARTITIONED|
-                                                                      -- SORT_GROUP_BY[$$721, $$722, $$723]  |PARTITIONED|
-                                                                              {
-                                                                                -- AGGREGATE  |LOCAL|
-                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                              }
-                                                                        -- 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 (test.tenk)  |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|
-                                                              -- SORT_GROUP_BY[$$10198, $$10199, $$10200]  |PARTITIONED|
-                                                                      {
-                                                                        -- AGGREGATE  |LOCAL|
-                                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                      }
-                                                                -- HASH_PARTITION_EXCHANGE [$$10198, $$10199, $$10200]  |PARTITIONED|
-                                                                  -- SORT_GROUP_BY[$$724, $$725, $$726]  |PARTITIONED|
-                                                                          {
-                                                                            -- AGGREGATE  |LOCAL|
-                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                          }
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- SORT_GROUP_BY[$$10196, $$10197, $$10198, $$10199]  |PARTITIONED|
+                                                                        {
+                                                                          -- AGGREGATE  |LOCAL|
+                                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                        }
+                                                                  -- HASH_PARTITION_EXCHANGE [$$10196, $$10197, $$10198, $$10199]  |PARTITIONED|
+                                                                    -- SORT_GROUP_BY[$$724, $$725, $$726, $$727]  |PARTITIONED|
+                                                                            {
+                                                                              -- AGGREGATE  |LOCAL|
+                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                            }
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                         -- ASSIGN  |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 (test.tenk)  |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|
-                                                          -- SORT_GROUP_BY[$$10202, $$10203]  |PARTITIONED|
-                                                                  {
-                                                                    -- AGGREGATE  |LOCAL|
-                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                  }
-                                                            -- HASH_PARTITION_EXCHANGE [$$10202, $$10203]  |PARTITIONED|
-                                                              -- SORT_GROUP_BY[$$727, $$728]  |PARTITIONED|
-                                                                      {
-                                                                        -- AGGREGATE  |LOCAL|
-                                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                      }
-                                                                -- 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|
+                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- SORT_GROUP_BY[$$10201, $$10202, $$10203]  |PARTITIONED|
+                                                                        {
+                                                                          -- AGGREGATE  |LOCAL|
+                                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                        }
+                                                                  -- HASH_PARTITION_EXCHANGE [$$10201, $$10202, $$10203]  |PARTITIONED|
+                                                                    -- SORT_GROUP_BY[$$728, $$729, $$730]  |PARTITIONED|
+                                                                            {
+                                                                              -- AGGREGATE  |LOCAL|
+                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                            }
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                         -- ASSIGN  |PARTITIONED|
                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                             -- REPLICATE  |PARTITIONED|
@@ -137,222 +76,197 @@
                                                                                     -- DATASOURCE_SCAN (test.tenk)  |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|
-                                                      -- SORT_GROUP_BY[$$10205, $$10206, $$10207]  |PARTITIONED|
-                                                              {
-                                                                -- AGGREGATE  |LOCAL|
-                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                              }
-                                                        -- HASH_PARTITION_EXCHANGE [$$10205, $$10206, $$10207]  |PARTITIONED|
-                                                          -- SORT_GROUP_BY[$$729, $$730, $$731]  |PARTITIONED|
-                                                                  {
-                                                                    -- AGGREGATE  |LOCAL|
-                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                  }
-                                                            -- 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|
-                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- DATASOURCE_SCAN (test.tenk)  |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|
-                                                  -- SORT_GROUP_BY[$$10209, $$10210]  |PARTITIONED|
-                                                          {
-                                                            -- AGGREGATE  |LOCAL|
-                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                          }
-                                                    -- HASH_PARTITION_EXCHANGE [$$10209, $$10210]  |PARTITIONED|
-                                                      -- SORT_GROUP_BY[$$732, $$733]  |PARTITIONED|
-                                                              {
-                                                                -- AGGREGATE  |LOCAL|
-                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                              }
-                                                        -- 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|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- DATASOURCE_SCAN (test.tenk)  |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|
-                                              -- SORT_GROUP_BY[$$10212, $$10213]  |PARTITIONED|
-                                                      {
-                                                        -- AGGREGATE  |LOCAL|
-                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                      }
-                                                -- HASH_PARTITION_EXCHANGE [$$10212, $$10213]  |PARTITIONED|
-                                                  -- SORT_GROUP_BY[$$734, $$735]  |PARTITIONED|
-                                                          {
-                                                            -- AGGREGATE  |LOCAL|
-                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                          }
-                                                    -- 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|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- DATASOURCE_SCAN (test.tenk)  |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|
-                                          -- SORT_GROUP_BY[$$10215]  |PARTITIONED|
-                                                  {
-                                                    -- AGGREGATE  |LOCAL|
-                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                  }
-                                            -- HASH_PARTITION_EXCHANGE [$$10215]  |PARTITIONED|
-                                              -- SORT_GROUP_BY[$$736]  |PARTITIONED|
-                                                      {
-                                                        -- AGGREGATE  |LOCAL|
-                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                      }
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                         -- ASSIGN  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- REPLICATE  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- SORT_GROUP_BY[$$10205, $$10206, $$10207]  |PARTITIONED|
+                                                                    {
+                                                                      -- AGGREGATE  |LOCAL|
+                                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                    }
+                                                              -- HASH_PARTITION_EXCHANGE [$$10205, $$10206, $$10207]  |PARTITIONED|
+                                                                -- SORT_GROUP_BY[$$731, $$732, $$733]  |PARTITIONED|
+                                                                        {
+                                                                          -- AGGREGATE  |LOCAL|
+                                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                        }
                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (test.tenk)  |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|
-                                  -- SORT_GROUP_BY[$$10217, $$10218]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- HASH_PARTITION_EXCHANGE [$$10217, $$10218]  |PARTITIONED|
-                                      -- SORT_GROUP_BY[$$737, $$738]  |PARTITIONED|
-                                              {
-                                                -- AGGREGATE  |LOCAL|
-                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                              }
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |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 (test.tenk)  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- SORT_GROUP_BY[$$10209, $$10210]  |PARTITIONED|
+                                                                {
+                                                                  -- AGGREGATE  |LOCAL|
+                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                }
+                                                          -- HASH_PARTITION_EXCHANGE [$$10209, $$10210]  |PARTITIONED|
+                                                            -- SORT_GROUP_BY[$$734, $$735]  |PARTITIONED|
+                                                                    {
+                                                                      -- AGGREGATE  |LOCAL|
+                                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                    }
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- ASSIGN  |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 (test.tenk)  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                 -- ASSIGN  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- REPLICATE  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- SORT_GROUP_BY[$$10212, $$10213, $$10214]  |PARTITIONED|
+                                                            {
+                                                              -- AGGREGATE  |LOCAL|
+                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                            }
+                                                      -- HASH_PARTITION_EXCHANGE [$$10212, $$10213, $$10214]  |PARTITIONED|
+                                                        -- SORT_GROUP_BY[$$736, $$737, $$738]  |PARTITIONED|
+                                                                {
+                                                                  -- AGGREGATE  |LOCAL|
+                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                }
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- DATASOURCE_SCAN (test.tenk)  |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|
-                              -- SORT_GROUP_BY[$$10220]  |PARTITIONED|
-                                      {
-                                        -- AGGREGATE  |LOCAL|
-                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                      }
-                                -- HASH_PARTITION_EXCHANGE [$$10220]  |PARTITIONED|
-                                  -- SORT_GROUP_BY[$$739]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ASSIGN  |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 (test.tenk)  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- ASSIGN  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- REPLICATE  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                -- SORT_GROUP_BY[$$10216, $$10217]  |PARTITIONED|
+                                                        {
+                                                          -- AGGREGATE  |LOCAL|
+                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                        }
+                                                  -- HASH_PARTITION_EXCHANGE [$$10216, $$10217]  |PARTITIONED|
+                                                    -- SORT_GROUP_BY[$$739, $$740]  |PARTITIONED|
+                                                            {
+                                                              -- AGGREGATE  |LOCAL|
+                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                            }
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (test.tenk)  |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|
-                          -- SORT_GROUP_BY[$$10222]  |PARTITIONED|
-                                  {
-                                    -- AGGREGATE  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                  }
-                            -- HASH_PARTITION_EXCHANGE [$$10222]  |PARTITIONED|
-                              -- SORT_GROUP_BY[$$740]  |PARTITIONED|
-                                      {
-                                        -- AGGREGATE  |LOCAL|
-                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                      }
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ASSIGN  |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 (test.tenk)  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- ASSIGN  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                                            -- SORT_GROUP_BY[$$10219, $$10220]  |PARTITIONED|
+                                                    {
+                                                      -- AGGREGATE  |LOCAL|
+                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                    }
+                                              -- HASH_PARTITION_EXCHANGE [$$10219, $$10220]  |PARTITIONED|
+                                                -- SORT_GROUP_BY[$$741, $$742]  |PARTITIONED|
+                                                        {
+                                                          -- AGGREGATE  |LOCAL|
+                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                        }
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ASSIGN  |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 (test.tenk)  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- SORT_GROUP_BY[$$10222]  |PARTITIONED|
+                                                {
+                                                  -- AGGREGATE  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                }
+                                          -- HASH_PARTITION_EXCHANGE [$$10222]  |PARTITIONED|
+                                            -- SORT_GROUP_BY[$$743]  |PARTITIONED|
+                                                    {
+                                                      -- AGGREGATE  |LOCAL|
+                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                    }
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |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 (test.tenk)  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$10224, $$10225]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- HASH_PARTITION_EXCHANGE [$$10224, $$10225]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$744, $$745]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |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 (test.tenk)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- SORT_GROUP_BY[$$10224]  |PARTITIONED|
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
-                          -- HASH_PARTITION_EXCHANGE [$$10224]  |PARTITIONED|
-                            -- SORT_GROUP_BY[$$741]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- SORT_GROUP_BY[$$10227]  |PARTITIONED|
                                     {
                                       -- AGGREGATE  |LOCAL|
                                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                                     }
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$10227]  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$746]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- REPLICATE  |PARTITIONED|
@@ -362,3 +276,54 @@
                                                   -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                       -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$10229]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- HASH_PARTITION_EXCHANGE [$$10229]  |PARTITIONED|
+                            -- SORT_GROUP_BY[$$747]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |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 (test.tenk)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$10231]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- HASH_PARTITION_EXCHANGE [$$10231]  |PARTITIONED|
+                          -- SORT_GROUP_BY[$$748]  |PARTITIONED|
+                                  {
+                                    -- AGGREGATE  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                  }
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |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 (test.tenk)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/listify-3.1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/listify-3.1.plan
index 85fa62d..d28f82a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/listify-3.1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/listify-3.1.plan
@@ -1,43 +1,36 @@
 -- DISTRIBUTE_RESULT  |LOCAL|
   -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-    -- STREAM_PROJECT  |LOCAL|
-      -- ASSIGN  |LOCAL|
+    -- ASSIGN  |LOCAL|
+      -- AGGREGATE  |LOCAL|
         -- AGGREGATE  |LOCAL|
-          -- AGGREGATE  |LOCAL|
-            -- STREAM_PROJECT  |LOCAL|
-              -- ASSIGN  |LOCAL|
+          -- ASSIGN  |LOCAL|
+            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+              -- NESTED_LOOP  |LOCAL|
                 -- ONE_TO_ONE_EXCHANGE  |LOCAL|
                   -- NESTED_LOOP  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                      -- UNNEST  |UNPARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                      -- NESTED_LOOP  |LOCAL|
-                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                          -- UNNEST  |UNPARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                      -- ASSIGN  |LOCAL|
                         -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                          -- STREAM_PROJECT  |LOCAL|
-                            -- ASSIGN  |LOCAL|
-                              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                                -- REPLICATE  |LOCAL|
-                                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                          -- REPLICATE  |LOCAL|
+                            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                              -- AGGREGATE  |LOCAL|
+                                -- ASSIGN  |LOCAL|
+                                  -- AGGREGATE  |LOCAL|
                                     -- AGGREGATE  |LOCAL|
-                                      -- STREAM_PROJECT  |LOCAL|
-                                        -- ASSIGN  |LOCAL|
-                                          -- AGGREGATE  |LOCAL|
-                                            -- AGGREGATE  |LOCAL|
-                                              -- UNNEST  |UNPARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                      -- STREAM_PROJECT  |LOCAL|
-                        -- ASSIGN  |LOCAL|
-                          -- STREAM_PROJECT  |LOCAL|
-                            -- ASSIGN  |LOCAL|
-                              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                                -- REPLICATE  |LOCAL|
-                                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                                    -- AGGREGATE  |LOCAL|
-                                      -- STREAM_PROJECT  |LOCAL|
-                                        -- ASSIGN  |LOCAL|
-                                          -- AGGREGATE  |LOCAL|
-                                            -- AGGREGATE  |LOCAL|
-                                              -- UNNEST  |UNPARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                      -- UNNEST  |UNPARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- ASSIGN  |LOCAL|
+                    -- ASSIGN  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                        -- REPLICATE  |LOCAL|
+                          -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                            -- AGGREGATE  |LOCAL|
+                              -- ASSIGN  |LOCAL|
+                                -- AGGREGATE  |LOCAL|
+                                  -- AGGREGATE  |LOCAL|
+                                    -- UNNEST  |UNPARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/sugar-06-distinct.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/sugar-06-distinct.plan
index ea1e178..a33ef53 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/sugar-06-distinct.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/sugar-06-distinct.plan
@@ -1,27 +1,25 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     -- STREAM_LIMIT  |UNPARTITIONED|
-      -- STREAM_PROJECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$$55(DESC) ]  |PARTITIONED|
-            -- STREAM_LIMIT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [topK: 3] [$$55(DESC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- PRE_CLUSTERED_GROUP_BY[$$53]  |PARTITIONED|
-                            {
-                              -- AGGREGATE  |LOCAL|
-                                -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                                  -- MICRO_STABLE_SORT [$$49(ASC)]  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                            }
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STABLE_SORT [$$53(ASC)]  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$59(DESC) ]  |PARTITIONED|
+          -- STREAM_LIMIT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [topK: 3] [$$59(DESC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- PRE_CLUSTERED_GROUP_BY[$$57]  |PARTITIONED|
+                          {
+                            -- AGGREGATE  |LOCAL|
+                              -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                                -- MICRO_STABLE_SORT [$$53(ASC)]  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                          }
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$57(ASC)]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$57]  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
                             -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN (gby.Employee)  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (gby.Employee)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.1.plan
index 8b2d960..4ee9d7f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.1.plan
@@ -4,71 +4,64 @@
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
         -- AGGREGATE  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$127, $$117][$$128, $$116]  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$127, $$117]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$134, $$124][$$135, $$123]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$134, $$124]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- HYBRID_HASH_JOIN [$$132][$$131]  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$132]  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$139][$$138]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$139]  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$117][$$115]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$117]  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$124][$$122]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$124]  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- HYBRID_HASH_JOIN [$$130][$$129]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- STREAM_SELECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$137][$$136]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$136]  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$129]  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$115]  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- HYBRID_HASH_JOIN [$$125][$$126]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$125]  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$132][$$133]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$132]  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
                                           -- ASSIGN  |PARTITIONED|
                                             -- STREAM_PROJECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$126]  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- STREAM_SELECT  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$131]  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.lineitem)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$128, $$116]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$138]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.lineitem)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$135, $$123]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.2.plan
index d56477e..f995ac2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.2.plan
@@ -4,32 +4,29 @@
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
         -- AGGREGATE  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$65][$$63]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$69][$$67]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- HYBRID_HASH_JOIN [$$65][$$64]  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$65]  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$64]  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$63]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$69][$$68]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$69]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$68]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$67]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.3.plan
index 5264d56..8f0ea7c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.3.plan
@@ -4,32 +4,29 @@
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
         -- AGGREGATE  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$68][$$66]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$72][$$70]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- HYBRID_HASH_JOIN [$$67][$$68]  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$67]  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$68]  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$66]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$71][$$72]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$71]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$72]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$70]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |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/hash-join-with-redundant-variable/hash-join-with-redundant-variable.4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.4.plan
index c898cde..4d8bf76 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hash-join-with-redundant-variable/hash-join-with-redundant-variable.4.plan
@@ -4,21 +4,19 @@
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
         -- AGGREGATE  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$49, $$56][$$52, $$53]  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$49, $$56]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$52, $$59][$$55, $$56]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$52, $$59]  |PARTITIONED|
                 -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$52, $$53]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                        -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$55, $$56]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |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/hints/broadcast_join_hint/broadcast_join_hint_1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_1.plan
index be7b38a..5467122 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_1.plan
@@ -1,23 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$39, $$41, $$43][$$40, $$42, $$44]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- HYBRID_HASH_JOIN [$$41, $$43, $$45][$$42, $$44, $$46]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- BROADCAST_EXCHANGE  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (test.t2)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.t2)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_2.plan
index be7b38a..5467122 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_2.plan
@@ -1,23 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$39, $$41, $$43][$$40, $$42, $$44]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- HYBRID_HASH_JOIN [$$41, $$43, $$45][$$42, $$44, $$46]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- BROADCAST_EXCHANGE  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (test.t2)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.t2)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_3.plan
index 552f7ef..98e6696 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_3.plan
@@ -1,23 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$39, $$42, $$43][$$40, $$41, $$44]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- HYBRID_HASH_JOIN [$$41, $$44, $$45][$$42, $$43, $$46]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- BROADCAST_EXCHANGE  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (test.t2)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.t2)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |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 fd34038..6848ad7 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
@@ -1,34 +1,31 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- HYBRID_HASH_JOIN [$$86][$$85]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- HYBRID_HASH_JOIN [$$86][$$85]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (tpch.orders)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (tpch.orders)  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$83][$$81]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (tpch.customer)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$81]  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.nation)  |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|
-                            -- HYBRID_HASH_JOIN [$$83][$$81]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (tpch.customer)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |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 c1e55d8..6c1b9e2 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
@@ -1,34 +1,31 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- HYBRID_HASH_JOIN [$$85][$$86]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- HYBRID_HASH_JOIN [$$85][$$86]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$83][$$81]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$83][$$81]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (tpch.customer)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (tpch.customer)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$81]  |PARTITIONED|
-                                -- DATASOURCE_SCAN (tpch.nation)  |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|
-                            -- DATASOURCE_SCAN (tpch.orders)  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$81]  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.nation)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |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/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 fd34038..6848ad7 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
@@ -1,34 +1,31 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- HYBRID_HASH_JOIN [$$86][$$85]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- HYBRID_HASH_JOIN [$$86][$$85]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (tpch.orders)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (tpch.orders)  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$83][$$81]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (tpch.customer)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$81]  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.nation)  |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|
-                            -- HYBRID_HASH_JOIN [$$83][$$81]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (tpch.customer)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |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 fd34038..6848ad7 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
@@ -1,34 +1,31 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- HYBRID_HASH_JOIN [$$86][$$85]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- HYBRID_HASH_JOIN [$$86][$$85]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (tpch.orders)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (tpch.orders)  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$83][$$81]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (tpch.customer)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$81]  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.nation)  |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|
-                            -- HYBRID_HASH_JOIN [$$83][$$81]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (tpch.customer)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |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 8f0ab1f..a27f96b 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
@@ -1,34 +1,31 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- HYBRID_HASH_JOIN [$$85][$$86]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- HYBRID_HASH_JOIN [$$85][$$86]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$83][$$81]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$83][$$81]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (tpch.customer)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (tpch.customer)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$81]  |PARTITIONED|
-                                -- DATASOURCE_SCAN (tpch.nation)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$86]  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (tpch.orders)  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$81]  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.nation)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$86]  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |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/hints/hashjoin_hint/hashjoin_hint_2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_2.plan
index 3e1214a..d495188 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
@@ -1,34 +1,31 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- HYBRID_HASH_JOIN [$$86][$$85]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$86]  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- HYBRID_HASH_JOIN [$$86][$$85]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$86]  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (tpch.orders)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (tpch.orders)  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$83][$$81]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (tpch.customer)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$81]  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.nation)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$83][$$81]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (tpch.customer)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |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 3e1214a..d495188 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
@@ -1,34 +1,31 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- HYBRID_HASH_JOIN [$$86][$$85]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$86]  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- HYBRID_HASH_JOIN [$$86][$$85]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$86]  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (tpch.orders)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (tpch.orders)  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$83][$$81]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (tpch.customer)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$81]  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.nation)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$83][$$81]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (tpch.customer)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |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 3e1214a..d495188 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
@@ -1,34 +1,31 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- HYBRID_HASH_JOIN [$$86][$$85]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$86]  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- HYBRID_HASH_JOIN [$$86][$$85]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$86]  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (tpch.orders)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (tpch.orders)  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$83][$$81]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (tpch.customer)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$81]  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.nation)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$83][$$81]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (tpch.customer)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |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 8f0ab1f..a27f96b 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
@@ -1,34 +1,31 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- HYBRID_HASH_JOIN [$$85][$$86]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- HYBRID_HASH_JOIN [$$85][$$86]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$83][$$81]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$83][$$81]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (tpch.customer)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (tpch.customer)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$81]  |PARTITIONED|
-                                -- DATASOURCE_SCAN (tpch.nation)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$86]  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (tpch.orders)  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$81]  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.nation)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$86]  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |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/hints/hashjoin_hint/hashjoin_hint_6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_6.plan
index 8f0ab1f..a27f96b 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
@@ -1,34 +1,31 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- HYBRID_HASH_JOIN [$$85][$$86]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- HYBRID_HASH_JOIN [$$85][$$86]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$83][$$81]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$83][$$81]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (tpch.customer)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (tpch.customer)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$81]  |PARTITIONED|
-                                -- DATASOURCE_SCAN (tpch.nation)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$86]  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (tpch.orders)  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$81]  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.nation)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$86]  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |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/index-through-object/index-through-object.1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.1.plan
index ff509f4..43ed493 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.1.plan
@@ -1,18 +1,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.10.plan
index f5ab2c2..d3ba232 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.10.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.10.plan
@@ -1,20 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$80]  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_SELECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$86(ASC)]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$86]  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.11.plan
index 6971ab4..3247fb6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.11.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.11.plan
@@ -2,38 +2,34 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- UNION_ALL  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
               -- STREAM_PROJECT  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                      -- STABLE_SORT [$$79(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$78(ASC)]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- BTREE_SEARCH (Test.Users.firstUsersNameIdx)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (Test.Users.firstUsersNameIdx)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
               -- STREAM_PROJECT  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                      -- STABLE_SORT [$$83(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$82(ASC)]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- BTREE_SEARCH (Test.Users.lastUsersNameIdx)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (Test.Users.lastUsersNameIdx)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.2.plan
index 90816ce..204b3ce 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.2.plan
@@ -1,18 +1,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$51(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.3.plan
index 6930560..39a3afb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.3.plan
@@ -1,18 +1,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$40(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.4.plan
index ff509f4..43ed493 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.4.plan
@@ -1,18 +1,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.5.plan
index f0f6a03..aefe73e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.5.plan
@@ -1,11 +1,10 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.6.plan
index b4aee6b..9c4deeb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.6.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.6.plan
@@ -1,20 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_SELECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$62(ASC)]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.7.plan
index f0f6a03..aefe73e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.7.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.7.plan
@@ -1,11 +1,10 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.8.plan
index 6930560..39a3afb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.8.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.8.plan
@@ -1,18 +1,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$40(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.9.plan
index 660af61..dfb15b1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.9.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.9.plan
@@ -1,39 +1,38 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$88][$$89]  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$88]  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$111(ASC)]  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$89]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- HYBRID_HASH_JOIN [$$88][$$89]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$88]  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
                 -- STREAM_SELECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$115(ASC)]  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$111(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$89]  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$115(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset.plan
index 9c4bd33..0fba5fc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset.plan
@@ -4,12 +4,11 @@
       -- INSERT_DELETE  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- MATERIALIZE  |PARTITIONED|
-            -- HASH_PARTITION_EXCHANGE [$$16]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$17]  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.myData)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |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/introhashpartitionmerge.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/introhashpartitionmerge.plan
index bff246a..1461b41 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/introhashpartitionmerge.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/introhashpartitionmerge.plan
@@ -6,19 +6,17 @@
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- HYBRID_HASH_JOIN [$$46][$$47]  |PARTITIONED|
               -- HASH_PARTITION_EXCHANGE [$$46]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (fuzzyjoin.TOKENSRANKEDADM)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$45(ASC)] HASH:[$$47]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- DATASOURCE_SCAN (fuzzyjoin.TOKENSRANKEDADM)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$45(ASC)] HASH:[$$47]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |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-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan
index 3003bc7..b0c4d3b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan
@@ -1,15 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |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-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan
index 3003bc7..b0c4d3b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan
@@ -1,15 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |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-complex/ngram-edit-distance-check-substring.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-substring.plan
index 94e20b4..3321d55 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-substring.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-substring.plan
@@ -1,16 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan
index 41c1583..fb9a209 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan
@@ -1,17 +1,15 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
           -- STREAM_PROJECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                  -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
+                      -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan
index 5c7a694..9feb07e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan
@@ -1,17 +1,15 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
           -- STREAM_PROJECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                  -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
+                      -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.keyword_index)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.keyword_index)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance-inline.plan
index 3c96aca..2c0f23e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance-inline.plan
@@ -2,61 +2,53 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- UNION_ALL  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$59(ASC)]  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_SELECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- REPLICATE  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- REPLICATE  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |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|
-      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- NESTED_LOOP  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
+                    -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$62(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- REPLICATE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- REPLICATE  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- REPLICATE  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |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|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- REPLICATE  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.plan
index 22ebacb..05daf89 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.plan
@@ -2,51 +2,44 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- UNION_ALL  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |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|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_SELECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- REPLICATE  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |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|
-      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- NESTED_LOOP  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$49(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- REPLICATE  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
+                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index)  |PARTITIONED|
+                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                  -- REPLICATE  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- REPLICATE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |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-join-noeqjoin/ngram-jaccard-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan
index f4bdc09..ec68814 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
@@ -1,34 +1,29 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$61][$$52]  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$61]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- HYBRID_HASH_JOIN [$$61][$$52]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$61]  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$64(ASC)]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                                        -- BROADCAST_EXCHANGE  |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|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$64(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |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-join-noeqjoin/olist-edit-distance-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance-inline.plan
index 5897e1f..0ffaa6f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance-inline.plan
@@ -2,61 +2,53 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- UNION_ALL  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$59(ASC)]  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
-                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_SELECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- REPLICATE  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- REPLICATE  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- NESTED_LOOP  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
+                    -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$62(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- REPLICATE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- REPLICATE  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- REPLICATE  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- REPLICATE  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard-inline.plan
index 66420f6..a8bd7bb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard-inline.plan
@@ -1,23 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$56(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
-                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$59(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |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-join-noeqjoin/ulist-jaccard-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard-inline.plan
index 66420f6..a8bd7bb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard-inline.plan
@@ -1,23 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$56(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
-                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$59(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |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-join-noeqjoin/word-jaccard-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan
index 6549e08..9cb276a 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
@@ -1,34 +1,29 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$61][$$52]  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$61]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- HYBRID_HASH_JOIN [$$61][$$52]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$61]  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$64(ASC)]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.keyword_index)  |PARTITIONED|
-                                        -- BROADCAST_EXCHANGE  |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|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$64(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.keyword_index)  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |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-join/issue741.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/issue741.plan
index 075b602..734201e 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
@@ -1,45 +1,39 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$60]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- STREAM_SELECT  |LOCAL|
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$60]  |PARTITIONED|
-                  -- STREAM_PROJECT  |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|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (test.TweetMessages)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$60]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$60]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- STREAM_SELECT  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$60]  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$71][$$60]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$71]  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.TweetMessages)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$60]  |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 [$$74(ASC)]  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$74(ASC)]  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.topicIIx)  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- STREAM_SELECT  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (test.TweetMessages)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.topicIIx)  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.TweetMessages)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |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 98a544e..416527f 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
@@ -1,78 +1,71 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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 [$$62(ASC), $$63(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$78][$$62]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |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 [$$62(ASC), $$63(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$78][$$62]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- REPLICATE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                        -- UNION_ALL  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
-                          -- UNION_ALL  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STABLE_SORT [$$86(ASC)]  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx)  |PARTITIONED|
-                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_SELECT  |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|
-                                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- NESTED_LOOP  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (test.TweetMessages)  |PARTITIONED|
+                                        -- STABLE_SORT [$$86(ASC)]  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_SELECT  |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|
+                                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx)  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_SELECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                    -- REPLICATE  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                         -- ASSIGN  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- REPLICATE  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- NESTED_LOOP  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test.TweetMessages)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- REPLICATE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |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/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 86e90b4..ddafba9 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
@@ -1,165 +1,152 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |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[$$62]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- STREAM_SELECT  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$62(ASC), $$63(ASC)]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$78][$$62]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |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[$$62]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- STREAM_SELECT  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$62(ASC), $$63(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$78][$$62]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
+                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                                    -- UNION_ALL  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                 -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
-                                      -- UNION_ALL  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- STREAM_SELECT  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- STABLE_SORT [$$86(ASC)]  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx)  |PARTITIONED|
-                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                  -- STREAM_SELECT  |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|
-                                                                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- ASSIGN  |PARTITIONED|
-                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- NESTED_LOOP  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (test.TweetMessages)  |PARTITIONED|
+                                                    -- STABLE_SORT [$$86(ASC)]  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_SELECT  |PARTITIONED|
+                                                        -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx)  |PARTITIONED|
+                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- REPLICATE  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- REPLICATE  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- NESTED_LOOP  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.TweetMessages)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- REPLICATE  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- REPLICATE  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$62]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$62(ASC), $$63(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$78][$$62]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                   -- REPLICATE  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                                              -- UNION_ALL  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
                                                       -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- REPLICATE  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- ASSIGN  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$62]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- STREAM_SELECT  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$62(ASC), $$63(ASC)]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
-                                        -- STREAM_PROJECT  |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|
-                                                      -- REPLICATE  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                           -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
-                                                -- UNION_ALL  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- STABLE_SORT [$$86(ASC)]  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx)  |PARTITIONED|
-                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                            -- STREAM_SELECT  |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|
-                                                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- NESTED_LOOP  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- DATASOURCE_SCAN (test.TweetMessages)  |PARTITIONED|
+                                                              -- STABLE_SORT [$$86(ASC)]  |PARTITIONED|
                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_SELECT  |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|
+                                                                  -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx)  |PARTITIONED|
+                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                      -- STREAM_SELECT  |PARTITIONED|
                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                          -- REPLICATE  |PARTITIONED|
                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                               -- ASSIGN  |PARTITIONED|
-                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- NESTED_LOOP  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (test.TweetMessages)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- REPLICATE  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- REPLICATE  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |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/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 34268a6..89bf4d3 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
@@ -1,48 +1,44 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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 [$$62(ASC), $$63(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$78][$$62]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |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 [$$62(ASC), $$63(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$78][$$62]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- REPLICATE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
-                          -- STREAM_SELECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
                             -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                    -- STABLE_SORT [$$82(ASC)]  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STABLE_SORT [$$82(ASC)]  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.topicKeywordIx)  |PARTITIONED|
-                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- REPLICATE  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.topicKeywordIx)  |PARTITIONED|
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- REPLICATE  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |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/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 0b3fbc2..72b7990 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
@@ -1,105 +1,98 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |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[$$62]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- STREAM_SELECT  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$62(ASC), $$63(ASC)]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
-                              -- STREAM_PROJECT  |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|
-                                            -- REPLICATE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
-                                      -- STREAM_SELECT  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STABLE_SORT [$$82(ASC)]  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.topicKeywordIx)  |PARTITIONED|
-                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- REPLICATE  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |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[$$62]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- STREAM_SELECT  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$62(ASC), $$63(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$62]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- STREAM_SELECT  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$62(ASC), $$63(ASC)]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$78][$$62]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- HYBRID_HASH_JOIN [$$78][$$62]  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
+                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STABLE_SORT [$$82(ASC)]  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.topicKeywordIx)  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- REPLICATE  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$62]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$62(ASC), $$63(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$78][$$62]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- REPLICATE  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- REPLICATE  |PARTITIONED|
+                                                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
-                                                -- STREAM_SELECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
                                                   -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                          -- STABLE_SORT [$$82(ASC)]  |PARTITIONED|
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- STABLE_SORT [$$82(ASC)]  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.topicKeywordIx)  |PARTITIONED|
-                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        -- ASSIGN  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- REPLICATE  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.topicKeywordIx)  |PARTITIONED|
+                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- REPLICATE  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |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/ngram-contains_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-contains_ps.plan
index b7351a2..903f710 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-contains_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-contains_ps.plan
@@ -1,54 +1,49 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-              -- RANGE_PARTITION_EXCHANGE [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-                -- FORWARD  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- REPLICATE  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |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|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |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|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
+                              -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
+                                  -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                                            -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                                                      -- BROADCAST_EXCHANGE  |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|
+                                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |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-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 801cb44..a2c86fa 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
@@ -1,55 +1,51 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
-                -- UNION_ALL  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |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|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index)  |PARTITIONED|
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_SELECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |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|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
+              -- UNION_ALL  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- NESTED_LOOP  |PARTITIONED|
+                        -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                            -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- REPLICATE  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index)  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- NESTED_LOOP  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |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 05a0087..37d392b 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
@@ -1,55 +1,51 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
-                -- UNION_ALL  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |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|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_SELECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |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|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
+              -- UNION_ALL  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- NESTED_LOOP  |PARTITIONED|
+                        -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                            -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- REPLICATE  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- NESTED_LOOP  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |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 4231828..4139775 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
@@ -1,59 +1,53 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$47]  |PARTITIONED|
-                -- UNION_ALL  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$62(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_SELECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- REPLICATE  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |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|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- NESTED_LOOP  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- REPLICATE  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$47]  |PARTITIONED|
+              -- UNION_ALL  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$62(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- REPLICATE  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- NESTED_LOOP  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- REPLICATE  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |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-join/ngram-edit-distance_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan
index 801cb44..a2c86fa 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
@@ -1,55 +1,51 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
-                -- UNION_ALL  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |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|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index)  |PARTITIONED|
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_SELECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |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|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
+              -- UNION_ALL  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- NESTED_LOOP  |PARTITIONED|
+                        -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                            -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- REPLICATE  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index)  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- NESTED_LOOP  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |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 05a0087..37d392b 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
@@ -1,55 +1,51 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
-                -- UNION_ALL  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |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|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_SELECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |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|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
+              -- UNION_ALL  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- NESTED_LOOP  |PARTITIONED|
+                        -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                            -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- REPLICATE  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- NESTED_LOOP  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |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 e549036..9c656dc 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
@@ -1,59 +1,53 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$46]  |PARTITIONED|
-                -- UNION_ALL  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$62(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_SELECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- REPLICATE  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |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|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- NESTED_LOOP  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- REPLICATE  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$46]  |PARTITIONED|
+              -- UNION_ALL  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$62(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- REPLICATE  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- NESTED_LOOP  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- REPLICATE  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |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-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 9b07d54..466f854 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
@@ -1,55 +1,51 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$34]  |PARTITIONED|
-                -- UNION_ALL  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |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|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index)  |PARTITIONED|
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_SELECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |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|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$34]  |PARTITIONED|
+              -- UNION_ALL  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- NESTED_LOOP  |PARTITIONED|
+                        -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                            -- STABLE_SORT [$$47(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- REPLICATE  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index)  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- NESTED_LOOP  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |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 a63ddbb..094ea28 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
@@ -1,55 +1,51 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$34]  |PARTITIONED|
-                -- UNION_ALL  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |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|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_SELECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |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|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$34]  |PARTITIONED|
+              -- UNION_ALL  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- NESTED_LOOP  |PARTITIONED|
+                        -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                            -- STABLE_SORT [$$47(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- REPLICATE  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- NESTED_LOOP  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |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 d9a606c..1c0ee3c 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
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$36]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$36]  |PARTITIONED|
+              -- STREAM_SELECT  |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|
-                        -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
+                        -- STABLE_SORT [$$49(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$49(ASC)]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |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|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |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-join/ngram-fuzzyeq-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan
index 58573b2..18170c0 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
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$36]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$36]  |PARTITIONED|
+              -- STREAM_SELECT  |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|
-                        -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                        -- STABLE_SORT [$$49(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$49(ASC)]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |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|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |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-join/ngram-jaccard-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan
index a12dce8..a9d8491 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
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$37]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$37]  |PARTITIONED|
+              -- STREAM_SELECT  |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|
-                        -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
+                        -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |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|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |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-join/ngram-jaccard-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan
index 8281cb4..caf734c 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
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$37]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$37]  |PARTITIONED|
+              -- STREAM_SELECT  |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|
-                        -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                        -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |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|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |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-join/ngram-jaccard-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan
index 5502ec5..e06ae9f 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
@@ -1,29 +1,26 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$49]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$49]  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                          -- STABLE_SORT [$$64(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$64(ASC)]  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                                    -- BROADCAST_EXCHANGE  |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|
+                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |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-join/ngram-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan
index a12dce8..a9d8491 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
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$37]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$37]  |PARTITIONED|
+              -- STREAM_SELECT  |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|
-                        -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
+                        -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |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|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |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-join/ngram-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan
index 8281cb4..caf734c 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
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$37]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$37]  |PARTITIONED|
+              -- STREAM_SELECT  |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|
-                        -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                        -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |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|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |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-join/ngram-jaccard_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan
index a05aede..a4041cd 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
@@ -1,29 +1,26 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$48]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$48]  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                          -- STABLE_SORT [$$64(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$64(ASC)]  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                                    -- BROADCAST_EXCHANGE  |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|
+                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |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-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 798833d..1fdd9ca 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
@@ -1,55 +1,51 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
-                -- UNION_ALL  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |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 [$$48(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index)  |PARTITIONED|
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_SELECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
+              -- UNION_ALL  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- NESTED_LOOP  |PARTITIONED|
+                        -- BTREE_SEARCH (test.Customers2.Customers2)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                            -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.Customers2)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- REPLICATE  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index)  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- NESTED_LOOP  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.Customers2)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |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 855017c..382f807 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
@@ -1,55 +1,51 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
-                -- UNION_ALL  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |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 [$$48(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_SELECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
+              -- UNION_ALL  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- NESTED_LOOP  |PARTITIONED|
+                        -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                            -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- REPLICATE  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- NESTED_LOOP  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |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 50468e9..ab7cb7c 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
@@ -1,59 +1,53 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$47]  |PARTITIONED|
-                -- UNION_ALL  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$62(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
-                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_SELECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- REPLICATE  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- NESTED_LOOP  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- REPLICATE  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$47]  |PARTITIONED|
+              -- UNION_ALL  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$62(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- REPLICATE  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- NESTED_LOOP  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- REPLICATE  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |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-join/olist-edit-distance_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan
index 798833d..1fdd9ca 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
@@ -1,55 +1,51 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
-                -- UNION_ALL  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |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 [$$48(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index)  |PARTITIONED|
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_SELECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
+              -- UNION_ALL  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- NESTED_LOOP  |PARTITIONED|
+                        -- BTREE_SEARCH (test.Customers2.Customers2)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                            -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.Customers2)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- REPLICATE  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index)  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- NESTED_LOOP  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.Customers2)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |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 855017c..382f807 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
@@ -1,55 +1,51 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
-                -- UNION_ALL  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |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 [$$48(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_SELECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
+              -- UNION_ALL  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- NESTED_LOOP  |PARTITIONED|
+                        -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                            -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- REPLICATE  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- NESTED_LOOP  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |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 4d65bc4..59bc5b8 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
@@ -1,59 +1,53 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$46]  |PARTITIONED|
-                -- UNION_ALL  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$62(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
-                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_SELECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- REPLICATE  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- NESTED_LOOP  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- REPLICATE  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$46]  |PARTITIONED|
+              -- UNION_ALL  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$62(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- REPLICATE  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- NESTED_LOOP  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- REPLICATE  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |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-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 f7a5011..e39f17b 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
@@ -1,55 +1,51 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$34]  |PARTITIONED|
-                -- UNION_ALL  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |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 [$$47(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index)  |PARTITIONED|
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_SELECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$34]  |PARTITIONED|
+              -- UNION_ALL  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- NESTED_LOOP  |PARTITIONED|
+                        -- BTREE_SEARCH (test.Customers2.Customers2)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                            -- STABLE_SORT [$$47(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.Customers2)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- REPLICATE  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index)  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- NESTED_LOOP  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.Customers2)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |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 b8e3d79..96186f4 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
@@ -1,55 +1,51 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$34]  |PARTITIONED|
-                -- UNION_ALL  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |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 [$$47(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_SELECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$34]  |PARTITIONED|
+              -- UNION_ALL  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- NESTED_LOOP  |PARTITIONED|
+                        -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                            -- STABLE_SORT [$$47(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- REPLICATE  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- NESTED_LOOP  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |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 c13d83b..336840b 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
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$34]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$34]  |PARTITIONED|
+              -- STREAM_SELECT  |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|
-                        -- BTREE_SEARCH (test.Customers2.Customers2)  |PARTITIONED|
+                        -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |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-join/olist-fuzzyeq-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan
index 1e7b2ca..a08ef41 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
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$34]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$34]  |PARTITIONED|
+              -- STREAM_SELECT  |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|
-                        -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
+                        -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |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-join/olist-jaccard-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan
index 41af3ae..572a9b9 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
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
+              -- STREAM_SELECT  |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|
-                        -- BTREE_SEARCH (test.Customers2.Customers2)  |PARTITIONED|
+                        -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |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-join/olist-jaccard-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan
index 6dfa31d..0452e66 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
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
+              -- STREAM_SELECT  |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|
-                        -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
+                        -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |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-join/olist-jaccard-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_04.plan
index 8c62a6d..f76eaa7 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
@@ -1,30 +1,26 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$47]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$59(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$47]  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$59(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |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-join/olist-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan
index 41af3ae..572a9b9 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
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
+              -- STREAM_SELECT  |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|
-                        -- BTREE_SEARCH (test.Customers2.Customers2)  |PARTITIONED|
+                        -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |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-join/olist-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan
index 6dfa31d..0452e66 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
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
+              -- STREAM_SELECT  |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|
-                        -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
+                        -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |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-join/olist-jaccard_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_04.plan
index 782070e..6b8b2f1 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
@@ -1,30 +1,26 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$46]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$59(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$46]  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$59(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |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-join/ulist-fuzzyeq-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan
index c13d83b..336840b 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
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$34]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$34]  |PARTITIONED|
+              -- STREAM_SELECT  |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|
-                        -- BTREE_SEARCH (test.Customers2.Customers2)  |PARTITIONED|
+                        -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |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-join/ulist-fuzzyeq-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan
index 1e7b2ca..a08ef41 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
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$34]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$34]  |PARTITIONED|
+              -- STREAM_SELECT  |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|
-                        -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
+                        -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |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-join/ulist-jaccard-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan
index 41af3ae..572a9b9 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
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
+              -- STREAM_SELECT  |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|
-                        -- BTREE_SEARCH (test.Customers2.Customers2)  |PARTITIONED|
+                        -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |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-join/ulist-jaccard-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan
index 6dfa31d..0452e66 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
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
+              -- STREAM_SELECT  |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|
-                        -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
+                        -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |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-join/ulist-jaccard-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_04.plan
index 8c62a6d..f76eaa7 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
@@ -1,30 +1,26 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$47]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$59(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$47]  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$59(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |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-join/ulist-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan
index 41af3ae..572a9b9 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
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
+              -- STREAM_SELECT  |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|
-                        -- BTREE_SEARCH (test.Customers2.Customers2)  |PARTITIONED|
+                        -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |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-join/ulist-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan
index 6dfa31d..0452e66 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
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$35]  |PARTITIONED|
+              -- STREAM_SELECT  |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|
-                        -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
+                        -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |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-join/ulist-jaccard_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_04.plan
index 782070e..6b8b2f1 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
@@ -1,30 +1,26 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$46]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$59(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$46]  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.Customers.Customers)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$59(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index)  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |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-join/word-fuzzyeq-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan
index b9091a3..a1a21d9 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
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$36]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$36]  |PARTITIONED|
+              -- STREAM_SELECT  |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|
-                        -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
+                        -- STABLE_SORT [$$49(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$49(ASC)]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.keyword_index)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |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|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.keyword_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |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-join/word-fuzzyeq-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan
index f364b67..025274c 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
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$36]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$36]  |PARTITIONED|
+              -- STREAM_SELECT  |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|
-                        -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                        -- STABLE_SORT [$$49(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$49(ASC)]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.keyword_index)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |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|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.keyword_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |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-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 744eb2f..792cf8f 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
@@ -1,41 +1,35 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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|
-                      -- ASSIGN  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- HYBRID_HASH_JOIN [$$69][$$56]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$69]  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$74(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgTextIx)  |PARTITIONED|
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$74(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgTextIx)  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- REPLICATE  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- REPLICATE  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |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 2663394..3d53c70 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
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$37]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$37]  |PARTITIONED|
+              -- STREAM_SELECT  |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|
-                        -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
+                        -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.keyword_index)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |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|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.keyword_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |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-join/word-jaccard-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan
index c7b7f6d..6a14525 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
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$37]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$37]  |PARTITIONED|
+              -- STREAM_SELECT  |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|
-                        -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                        -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.keyword_index)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |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|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.keyword_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |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-join/word-jaccard-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan
index a7f501c..c48868f 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
@@ -1,29 +1,26 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$49]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$49]  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                          -- STABLE_SORT [$$64(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$64(ASC)]  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.keyword_index)  |PARTITIONED|
-                                    -- BROADCAST_EXCHANGE  |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|
+                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.keyword_index)  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |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-join/word-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan
index 2663394..3d53c70 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
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$37]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$37]  |PARTITIONED|
+              -- STREAM_SELECT  |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|
-                        -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
+                        -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.keyword_index)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |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|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.keyword_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |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-join/word-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan
index c7b7f6d..6a14525 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
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$37]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$37]  |PARTITIONED|
+              -- STREAM_SELECT  |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|
-                        -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                        -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.keyword_index)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |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|
+                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.keyword_index)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |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-join/word-jaccard_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan
index bfea2fb..e56f7a6 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
@@ -1,29 +1,26 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$48]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$48]  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                          -- STABLE_SORT [$$64(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$64(ASC)]  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.keyword_index)  |PARTITIONED|
-                                    -- BROADCAST_EXCHANGE  |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|
+                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.keyword_index)  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |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/join-singletonbranch-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-singletonbranch-2.plan
index 21d633b..cfa80ac 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-singletonbranch-2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-singletonbranch-2.plan
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- NESTED_LOOP  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.Bar)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- BROADCAST_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- NESTED_LOOP  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (test.Bar)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- BROADCAST_EXCHANGE  |PARTITIONED|
+              -- ASSIGN  |UNPARTITIONED|
                 -- ASSIGN  |UNPARTITIONED|
-                  -- STREAM_PROJECT  |UNPARTITIONED|
-                    -- ASSIGN  |UNPARTITIONED|
-                      -- AGGREGATE  |UNPARTITIONED|
-                        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                          -- AGGREGATE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (test.Foo)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.Foo)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-singletonbranch-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-singletonbranch-3.plan
index bb190b0..561e6d5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-singletonbranch-3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-singletonbranch-3.plan
@@ -1,23 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- DATASOURCE_SCAN (test.Bar)  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- ASSIGN  |UNPARTITIONED|
-                      -- STREAM_PROJECT  |UNPARTITIONED|
-                        -- ASSIGN  |UNPARTITIONED|
-                          -- AGGREGATE  |UNPARTITIONED|
-                            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                              -- AGGREGATE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (test.Foo)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN (test.Bar)  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |UNPARTITIONED|
+                  -- ASSIGN  |UNPARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN (test.Foo)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-singletonbranch.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-singletonbranch.plan
index de98287..0c6cf9b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-singletonbranch.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-singletonbranch.plan
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- NESTED_LOOP  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.Bar)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |UNPARTITIONED|
-                  -- ASSIGN  |UNPARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- AGGREGATE  |UNPARTITIONED|
-                        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                          -- AGGREGATE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (test.Foo)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- NESTED_LOOP  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (test.Bar)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- BROADCAST_EXCHANGE  |PARTITIONED|
+              -- ASSIGN  |UNPARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.Foo)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |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 6bae2cc..bafb0e3 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
@@ -1,22 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$39, $$44, $$46][$$41, $$45, $$42]  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$39, $$44, $$46]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- HYBRID_HASH_JOIN [$$39, $$44, $$46][$$41, $$45, $$42]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$39, $$44, $$46]  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (`join-super-key_1`.LineItems)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (`join-super-key_1`.LineItems)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$41, $$45, $$42]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$41, $$45, $$42]  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (`join-super-key_1`.PartSupp)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (`join-super-key_1`.PartSupp)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |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 da9d41a..cc2777f 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
@@ -1,22 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- HYBRID_HASH_JOIN [$$39, $$45, $$40][$$41, $$44, $$46]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$39, $$45, $$40]  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (`join-super-key_01`.PartSupp)  |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 [$$41, $$44, $$46]  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$41, $$44, $$46]  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (`join-super-key_01`.LineItems)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (`join-super-key_01`.LineItems)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/fnds_join_ds.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/fnds_join_ds.plan
index a561ca4..75fe2a1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/fnds_join_ds.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/fnds_join_ds.plan
@@ -1,24 +1,21 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- HYBRID_HASH_JOIN [$$48][$$49]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- HYBRID_HASH_JOIN [$$51][$$52]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$51]  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (asterix.tpcds-datagen.customer_address.1.0)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (asterix.tpcds-datagen.customer_address.1.0)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$49]  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (test.t1)  |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/joins/inner_right_corr.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/inner_right_corr.plan
index 8327ffc..409f99b 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
@@ -1,55 +1,51 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- PRE_CLUSTERED_GROUP_BY[$$76]  |PARTITIONED|
-                    {
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$76]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
                       -- AGGREGATE  |LOCAL|
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                    }
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$76(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- HYBRID_HASH_JOIN [$$76][$$92]  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$76]  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$92]  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                        -- STREAM_SELECT  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$76(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$76][$$92]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$76]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$92]  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- UNNEST  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
-                                -- UNNEST  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- SUBPLAN  |PARTITIONED|
-                                            {
-                                              -- AGGREGATE  |LOCAL|
-                                                -- STREAM_SELECT  |LOCAL|
-                                                  -- ASSIGN  |LOCAL|
-                                                    -- UNNEST  |LOCAL|
-                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                            }
+                                -- SUBPLAN  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- ASSIGN  |LOCAL|
+                                                -- UNNEST  |LOCAL|
+                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- NESTED_LOOP  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- NESTED_LOOP  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
+                                            -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.t2)  |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|
-                                                    -- DATASOURCE_SCAN (test.t2)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |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 a9d6547..1761df7 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
@@ -4,34 +4,31 @@
       -- INSERT_DELETE  |PARTITIONED|
         -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
           -- ASSIGN  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$40]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- STREAM_SELECT  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- PRE_CLUSTERED_GROUP_BY[$$40]  |PARTITIONED|
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- STREAM_SELECT  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$40(ASC)]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$40]  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
+                        -- STABLE_SORT [$$40(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$40]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$43][$$41]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- HYBRID_HASH_JOIN [$$43][$$41]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (test.tweetDataset)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (test.tweetDataset)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.countryDataset)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (test.countryDataset)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nlj_partitioning_property_1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nlj_partitioning_property_1.plan
index f5d57d5..576f967 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nlj_partitioning_property_1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nlj_partitioning_property_1.plan
@@ -1,29 +1,28 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- HYBRID_HASH_JOIN [$$71][$$73]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (tpch.Supplier)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- NESTED_LOOP  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (tpch.Partsupp)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (tpch.Part)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- HYBRID_HASH_JOIN [$$76][$$78]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (tpch.Supplier)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- NESTED_LOOP  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (tpch.Partsupp)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (tpch.Part)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nlj_partitioning_property_2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nlj_partitioning_property_2.plan
index 4cd1213..7fa1d25 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nlj_partitioning_property_2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nlj_partitioning_property_2.plan
@@ -1,29 +1,28 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- NESTED_LOOP  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (tpch.Supplier)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- NESTED_LOOP  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (tpch.Partsupp)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (tpch.Part)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (tpch.Supplier)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- NESTED_LOOP  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (tpch.Partsupp)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (tpch.Part)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/query-ASTERIXDB-2986.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/query-ASTERIXDB-2986.plan
index 4bbdb3e..499b6f8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/query-ASTERIXDB-2986.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/query-ASTERIXDB-2986.plan
@@ -1,39 +1,35 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- NESTED_LOOP  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$78][$$77]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$82][$$81]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$82]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test1.orders)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test1.orders)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$77]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test1.lineitem)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$81]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test1.lineitem)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (test1.customer)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (test1.customer)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/leftouterjoin/loj-03-no-listify.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/leftouterjoin/loj-03-no-listify.plan
index 6d569dc..bbf6c62 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/leftouterjoin/loj-03-no-listify.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/leftouterjoin/loj-03-no-listify.plan
@@ -1,36 +1,59 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$taskId(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$taskId(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- HYBRID_HASH_JOIN [$$taskId][$$taskId]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- HYBRID_HASH_JOIN [$$taskId][$$taskId]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$taskId][$$taskId]  |PARTITIONED|
-                                    -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$taskId(ASC)] HASH:[$$taskId]  |PARTITIONED|
-                                      -- SORT_GROUP_BY[$$279]  |PARTITIONED|
-                                              {
-                                                -- AGGREGATE  |LOCAL|
-                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                              }
-                                        -- HASH_PARTITION_EXCHANGE [$$279]  |PARTITIONED|
-                                          -- SORT_GROUP_BY[$$242]  |PARTITIONED|
-                                                  {
-                                                    -- AGGREGATE  |LOCAL|
-                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                  }
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$taskId(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$taskId(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- HYBRID_HASH_JOIN [$$taskId][$$taskId]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$taskId][$$taskId]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$taskId][$$taskId]  |PARTITIONED|
+                                  -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$taskId(ASC)] HASH:[$$taskId]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$295]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- HASH_PARTITION_EXCHANGE [$$295]  |PARTITIONED|
+                                        -- SORT_GROUP_BY[$$258]  |PARTITIONED|
+                                                {
+                                                  -- AGGREGATE  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                }
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- REPLICATE  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (test.tasks)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- SORT_GROUP_BY[$$297]  |PARTITIONED|
+                                                {
+                                                  -- AGGREGATE  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                }
+                                          -- HASH_PARTITION_EXCHANGE [$$297]  |PARTITIONED|
+                                            -- SORT_GROUP_BY[$$259]  |PARTITIONED|
+                                                    {
+                                                      -- AGGREGATE  |LOCAL|
+                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                    }
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_SELECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                     -- REPLICATE  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -40,81 +63,22 @@
                                                               -- DATASOURCE_SCAN (test.tasks)  |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|
-                                            -- SORT_GROUP_BY[$$281]  |PARTITIONED|
-                                                    {
-                                                      -- AGGREGATE  |LOCAL|
-                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                    }
-                                              -- HASH_PARTITION_EXCHANGE [$$281]  |PARTITIONED|
-                                                -- SORT_GROUP_BY[$$243]  |PARTITIONED|
-                                                        {
-                                                          -- AGGREGATE  |LOCAL|
-                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                        }
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- REPLICATE  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (test.tasks)  |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|
-                                    -- SORT_GROUP_BY[$$283]  |PARTITIONED|
-                                            {
-                                              -- AGGREGATE  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                            }
-                                      -- HASH_PARTITION_EXCHANGE [$$283]  |PARTITIONED|
-                                        -- SORT_GROUP_BY[$$244]  |PARTITIONED|
-                                                {
-                                                  -- AGGREGATE  |LOCAL|
-                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                }
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- REPLICATE  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- DATASOURCE_SCAN (test.tasks)  |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|
-                            -- SORT_GROUP_BY[$$285]  |PARTITIONED|
-                                    {
-                                      -- AGGREGATE  |LOCAL|
-                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                    }
-                              -- HASH_PARTITION_EXCHANGE [$$285]  |PARTITIONED|
-                                -- SORT_GROUP_BY[$$245]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$299]  |PARTITIONED|
                                         {
                                           -- AGGREGATE  |LOCAL|
                                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                                         }
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- STREAM_SELECT  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$299]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$260]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
                                           -- ASSIGN  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                               -- REPLICATE  |PARTITIONED|
@@ -125,3 +89,29 @@
                                                         -- DATASOURCE_SCAN (test.tasks)  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$301]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- HASH_PARTITION_EXCHANGE [$$301]  |PARTITIONED|
+                            -- SORT_GROUP_BY[$$261]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- REPLICATE  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.tasks)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |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 18934ba..79d50d8 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
@@ -1,39 +1,34 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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|
-                  -- NESTED_LOOP  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |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|
-                                      -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+    -- ASSIGN  |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|
+                -- NESTED_LOOP  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$136][$$137]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$136]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/leftouterjoin/query-ASTERIXDB-2988.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/leftouterjoin/query-ASTERIXDB-2988.plan
index 96fbf3a..8faa38b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/leftouterjoin/query-ASTERIXDB-2988.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/leftouterjoin/query-ASTERIXDB-2988.plan
@@ -1,20 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$r1(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$r1(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- HYBRID_HASH_JOIN [$$r1][$$32]  |PARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$r1]  |PARTITIONED|
-                      -- UNNEST  |UNPARTITIONED|
-                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$32]  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (test.ds1)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$r1(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$r1(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- HYBRID_HASH_JOIN [$$r1][$$34]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$r1]  |PARTITIONED|
+                    -- UNNEST  |UNPARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |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/loj-core.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-core.plan
index 677ef19..a88c8e6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-core.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-core.plan
@@ -1,24 +1,22 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$42(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- HYBRID_HASH_JOIN [$$44][$$45]  |PARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$44]  |PARTITIONED|
+    -- ASSIGN  |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 [$$47][$$48]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$47]  |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 [$$48]  |PARTITIONED|
+                    -- ASSIGN  |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 [$$45]  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-core_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-core_ps.plan
index 0061daa..a846224 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-core_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-core_ps.plan
@@ -1,53 +1,50 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$42(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$45(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 [$$47][$$48]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$47]  |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 [$$48]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- HYBRID_HASH_JOIN [$$44][$$45]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$44]  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |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 [$$45]  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- HYBRID_HASH_JOIN [$$44][$$45]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$44]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$47][$$48]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$47]  |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 [$$48]  |PARTITIONED|
+                                      -- ASSIGN  |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 [$$45]  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-sugar.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-sugar.plan
index 1dd720b..f754844 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-sugar.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-sugar.plan
@@ -1,24 +1,22 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$33(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- HYBRID_HASH_JOIN [$$35][$$36]  |PARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
+    -- ASSIGN  |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|
+                -- HYBRID_HASH_JOIN [$$37][$$38]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$37]  |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 [$$38]  |PARTITIONED|
+                    -- ASSIGN  |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 [$$36]  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-sugar_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-sugar_ps.plan
index d0c36bc..8f5e3eb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-sugar_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-sugar_ps.plan
@@ -1,53 +1,50 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$33(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$35(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 [$$37][$$38]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$37]  |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 [$$38]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- HYBRID_HASH_JOIN [$$35][$$36]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |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 [$$36]  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- HYBRID_HASH_JOIN [$$35][$$36]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$37][$$38]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$37]  |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 [$$38]  |PARTITIONED|
+                                      -- ASSIGN  |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 [$$36]  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |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 5d1ba2b..bbfc442 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
@@ -1,30 +1,29 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- PRE_CLUSTERED_GROUP_BY[$$49, $$50]  |PARTITIONED|
-                    {
-                      -- AGGREGATE  |LOCAL|
-                        -- STREAM_SELECT  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                    }
-              -- 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 [$$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|
-                          -- HASH_PARTITION_EXCHANGE [$$54, $$51, $$52]  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (`loj-super-key_01`.PartSupp)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$49, $$50]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- 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 [$$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|
+                        -- HASH_PARTITION_EXCHANGE [$$54, $$51, $$52]  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (`loj-super-key_01`.PartSupp)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |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 ccad49d..f42c8aa 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
@@ -1,31 +1,30 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- PRE_CLUSTERED_GROUP_BY[$$49, $$50]  |PARTITIONED|
-                    {
-                      -- AGGREGATE  |LOCAL|
-                        -- STREAM_SELECT  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                    }
-              -- 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 [$$54, $$49, $$50][$$53, $$51, $$57]  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$54, $$49, $$50]  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$49, $$50]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- 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 [$$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 [$$53, $$51, $$57]  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (`loj-super-key_02`.PartSupp)  |PARTITIONED|
+                                -- DATASOURCE_SCAN (`loj-super-key_02`.LineItems)  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$53, $$51, $$57]  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (`loj-super-key_02`.LineItems)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_1.plan
index cdbedfd..849a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_1.plan
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
               -- STREAM_PROJECT  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.ds)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.ds)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_2.plan
index a23a25f..b679b2c6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_2.plan
@@ -1,11 +1,10 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- BTREE_SEARCH (test.ds.primary_idx_ds)  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH (test.ds.primary_idx_ds)  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_3.plan
index a23a25f..b679b2c6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_3.plan
@@ -1,11 +1,10 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- BTREE_SEARCH (test.ds.primary_idx_ds)  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH (test.ds.primary_idx_ds)  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_4.plan
index cdbedfd..849a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_4.plan
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
               -- STREAM_PROJECT  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.ds)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.ds)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_01.plan
index 2561ec6..9dbe5c2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_01.plan
@@ -1,10 +1,9 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- DATASOURCE_SCAN (test.DS1)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |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_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_02.plan
index dad3cdc..cb90155 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_02.plan
@@ -1,10 +1,9 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- BTREE_SEARCH (test.DS1.DS1)  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- BTREE_SEARCH (test.DS1.DS1)  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_03.plan
index 0d81837..6f90d10 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_03.plan
@@ -1,10 +1,9 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- BTREE_SEARCH (test.DS2.DS2)  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- BTREE_SEARCH (test.DS2.DS2)  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_04.plan
index ef392eb..41bac78 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_04.plan
@@ -1,11 +1,9 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN (test.DS2)  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- DATASOURCE_SCAN (test.DS2)  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |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 2e2b4c8..8e5ac32 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
@@ -1,24 +1,23 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$31][$$36]  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$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 [$$36]  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (test.DS2)  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.DS1)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |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_06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_06.plan
index 326507f..03dbd6a 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
@@ -1,19 +1,18 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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|
-                        -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.DS2)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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|
+                      -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |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 fafbbb2..b78648b 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
@@ -1,20 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$$33(ASC) ]  |PARTITIONED|
-            -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |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|
-                                  -- DATASOURCE_SCAN (test.DS1)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$33(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |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|
+                                -- 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_08.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_08.plan
index dc7178d..f591cc0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_08.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_08.plan
@@ -1,17 +1,16 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- BTREE_SEARCH (test.DS1.DS1)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.DS1.id_sec_idx)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.DS1.DS1)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.DS1.id_sec_idx)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_09.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_09.plan
index dad3cdc..cb90155 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_09.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_09.plan
@@ -1,10 +1,9 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- BTREE_SEARCH (test.DS1.DS1)  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- BTREE_SEARCH (test.DS1.DS1)  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_10.plan
index 514628f..3caccff 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_10.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_10.plan
@@ -1,24 +1,23 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$$29(ASC) ]  |PARTITIONED|
-            -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (test.DS1.DS1)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$37(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.DS1.id_sec_idx)  |PARTITIONED|
-                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.DS2)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$31(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (test.DS1.DS1)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.DS1.id_sec_idx)  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |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_11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_11.plan
index 326507f..03dbd6a 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
@@ -1,19 +1,18 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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|
-                        -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.DS2)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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|
+                      -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |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/with_clause_meta.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/with_clause_meta.plan
index d0bcfb4..2fcabd6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/with_clause_meta.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/with_clause_meta.plan
@@ -1,44 +1,37 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     -- STREAM_LIMIT  |UNPARTITIONED|
-      -- STREAM_PROJECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$#1(ASC) ]  |PARTITIONED|
-            -- STREAM_LIMIT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [topK: 5] [$#1(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$#1(ASC) ]  |PARTITIONED|
+          -- STREAM_LIMIT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [topK: 5] [$#1(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- UNNEST  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- UNNEST  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- PRE_CLUSTERED_GROUP_BY[$$111]  |PARTITIONED|
-                                            {
-                                              -- AGGREGATE  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                            }
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STABLE_SORT [$$111(ASC)]  |PARTITIONED|
-                                          -- HASH_PARTITION_EXCHANGE [$$111]  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- HYBRID_HASH_JOIN [$$92][$$115]  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (test.ds)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                            -- UNNEST  |UNPARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_CLUSTERED_GROUP_BY[$$119]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$119(ASC)]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$119]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$100][$$123]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (test.ds)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/micro_external_sort/micro_external_sort.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/micro_external_sort/micro_external_sort.plan
index 4f8ff43..ecff808 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/micro_external_sort/micro_external_sort.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/micro_external_sort/micro_external_sort.plan
@@ -1,27 +1,25 @@
 -- DISTRIBUTE_RESULT  |LOCAL|
   -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-    -- STREAM_PROJECT  |LOCAL|
-      -- ASSIGN  |LOCAL|
-        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-          -- PRE_CLUSTERED_GROUP_BY[$$145]  |LOCAL|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- MICRO_STABLE_SORT [$$149(ASC)]  |LOCAL|
-                        -- MICRO_PRE_CLUSTERED_GROUP_BY[$$146]  |LOCAL|
-                                {
+    -- ASSIGN  |LOCAL|
+      -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+        -- PRE_CLUSTERED_GROUP_BY[$$163]  |LOCAL|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- MICRO_STABLE_SORT [$$167(ASC)]  |LOCAL|
+                      -- MICRO_PRE_CLUSTERED_GROUP_BY[$$164]  |LOCAL|
+                              {
+                                -- AGGREGATE  |LOCAL|
                                   -- AGGREGATE  |LOCAL|
-                                    -- AGGREGATE  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
-                          -- MICRO_STABLE_SORT [$$146(ASC)]  |LOCAL|
-                            -- ASSIGN  |LOCAL|
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-              -- STABLE_SORT [$$145(ASC)]  |LOCAL|
-                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                  -- STREAM_PROJECT  |UNPARTITIONED|
-                    -- ASSIGN  |UNPARTITIONED|
-                      -- ASSIGN  |UNPARTITIONED|
-                        -- UNNEST  |UNPARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
\ No newline at end of file
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- MICRO_STABLE_SORT [$$164(ASC)]  |LOCAL|
+                          -- ASSIGN  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+            -- STABLE_SORT [$$163(ASC)]  |LOCAL|
+              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                -- ASSIGN  |UNPARTITIONED|
+                  -- ASSIGN  |UNPARTITIONED|
+                    -- UNNEST  |UNPARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
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 9d2ca53..9887b10 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
@@ -1,15 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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|
-              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
-                -- DATASOURCE_SCAN (Facebook.Friendship)  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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|
+            -- 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/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.plan
index 8fd6f04..d41058b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.plan
@@ -1,86 +1,77 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$65(ASC)]  |PARTITIONED|
-              -- RANGE_PARTITION_EXCHANGE [$$65(ASC)]  |PARTITIONED|
-                -- FORWARD  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- REPLICATE  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- PRE_CLUSTERED_GROUP_BY[$$52]  |PARTITIONED|
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- STREAM_SELECT  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$67(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$67(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$54]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$54(ASC), $$55(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$72(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$52(ASC), $$53(ASC)]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$54]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$54(ASC), $$55(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STABLE_SORT [$$70(ASC)]  |PARTITIONED|
+                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- STABLE_SORT [$$72(ASC)]  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
                                                                     -- ASSIGN  |PARTITIONED|
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                         -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                             -- ASSIGN  |PARTITIONED|
                                                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$52]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$52(ASC), $$53(ASC)]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
-                                          -- STREAM_SELECT  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- STABLE_SORT [$$70(ASC)]  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.plan
index 9857dae..1c961a8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.plan
@@ -1,86 +1,77 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$73(ASC)]  |PARTITIONED|
-              -- RANGE_PARTITION_EXCHANGE [$$73(ASC)]  |PARTITIONED|
-                -- FORWARD  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- REPLICATE  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- PRE_CLUSTERED_GROUP_BY[$$58]  |PARTITIONED|
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- STREAM_SELECT  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$75(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$75(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$60]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$60(ASC), $$61(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$60]  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$58(ASC), $$59(ASC)]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$60]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$60(ASC), $$61(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$60]  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STABLE_SORT [$$78(ASC)]  |PARTITIONED|
+                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
                                                                     -- ASSIGN  |PARTITIONED|
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                         -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                             -- ASSIGN  |PARTITIONED|
                                                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$58]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$58(ASC), $$59(ASC)]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-                                          -- STREAM_SELECT  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- STABLE_SORT [$$78(ASC)]  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- ASSIGN  |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 0aa8dc4..3920a9f 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
@@ -1,190 +1,167 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |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[$$74]  |PARTITIONED|
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- STREAM_SELECT  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$74(ASC), $$75(ASC)]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
-                                -- STREAM_PROJECT  |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|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- REPLICATE  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
-                                        -- UNION_ALL  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- STABLE_SORT [$$107(ASC)]  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx)  |PARTITIONED|
-                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                        -- STREAM_SELECT  |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|
-                                                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- NESTED_LOOP  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- DATASOURCE_SCAN (test.TweetMessages)  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_SELECT  |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|
-                                                                          -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |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[$$74]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$74(ASC), $$75(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$74]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$74(ASC), $$75(ASC)]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$95][$$74]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$95]  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- REPLICATE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
+                                      -- UNION_ALL  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STABLE_SORT [$$107(ASC)]  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx)  |PARTITIONED|
+                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- REPLICATE  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$95][$$74]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$95]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                              -- NESTED_LOOP  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
                                                     -- ASSIGN  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN (test.TweetMessages)  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- REPLICATE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                -- REPLICATE  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$74]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$74(ASC), $$75(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$95][$$74]  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$95]  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
+                                                -- UNION_ALL  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- STABLE_SORT [$$107(ASC)]  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx)  |PARTITIONED|
+                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- NESTED_LOOP  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- DATASOURCE_SCAN (test.TweetMessages)  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- REPLICATE  |PARTITIONED|
                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                     -- ASSIGN  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
-                                                  -- UNION_ALL  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- STREAM_SELECT  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- STABLE_SORT [$$107(ASC)]  |PARTITIONED|
+                                                                          -- REPLICATE  |PARTITIONED|
                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx)  |PARTITIONED|
-                                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                  -- STREAM_SELECT  |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|
-                                                                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                            -- ASSIGN  |PARTITIONED|
-                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- NESTED_LOOP  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ASSIGN  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- DATASOURCE_SCAN (test.TweetMessages)  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                              -- STREAM_SELECT  |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|
-                                                                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- ASSIGN  |PARTITIONED|
-                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- ASSIGN  |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-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 f9d1b9d..2ca13b1 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
@@ -1,120 +1,105 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |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[$$74]  |PARTITIONED|
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- STREAM_SELECT  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$74(ASC), $$75(ASC)]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
-                                -- STREAM_PROJECT  |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|
+        -- ONE_TO_ONE_EXCHANGE  |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[$$74]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$74(ASC), $$75(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$95][$$74]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$95]  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- REPLICATE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
                                           -- ASSIGN  |PARTITIONED|
                                             -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- REPLICATE  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STABLE_SORT [$$101(ASC)]  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.topicKeywordIx)  |PARTITIONED|
+                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- REPLICATE  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$74]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$74(ASC), $$75(ASC)]  |PARTITIONED|
                                       -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
-                                          -- STREAM_SELECT  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$95][$$74]  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$95]  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
                                                   -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- STABLE_SORT [$$101(ASC)]  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.topicKeywordIx)  |PARTITIONED|
-                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                      -- ASSIGN  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- REPLICATE  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$74]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$74(ASC), $$75(ASC)]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$95][$$74]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$95]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
+                                                -- STREAM_SELECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
                                                     -- ASSIGN  |PARTITIONED|
                                                       -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- REPLICATE  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- ASSIGN  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- STABLE_SORT [$$101(ASC)]  |PARTITIONED|
                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- STABLE_SORT [$$101(ASC)]  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.topicKeywordIx)  |PARTITIONED|
-                                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.topicKeywordIx)  |PARTITIONED|
+                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- REPLICATE  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                     -- ASSIGN  |PARTITIONED|
-                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- REPLICATE  |PARTITIONED|
-                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
index ee5703a..4a08ca9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
@@ -1,92 +1,81 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$74(ASC)]  |PARTITIONED|
-              -- RANGE_PARTITION_EXCHANGE [$$74(ASC)]  |PARTITIONED|
-                -- FORWARD  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- REPLICATE  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- PRE_CLUSTERED_GROUP_BY[$$61]  |PARTITIONED|
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- STREAM_SELECT  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$61(ASC), $$67(ASC)]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$61]  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$77(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$77(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$64]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$64(ASC), $$70(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$64]  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$89(ASC)]  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STABLE_SORT [$$86(ASC)]  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
-                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- ASSIGN  |PARTITIONED|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                      -- ASSIGN  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$61]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$61(ASC), $$67(ASC)]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$61]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- STREAM_SELECT  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- STABLE_SORT [$$86(ASC)]  |PARTITIONED|
+                                                    -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
-                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$64]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$64(ASC), $$70(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$64]  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STABLE_SORT [$$89(ASC)]  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
+                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                 -- ASSIGN  |PARTITIONED|
-                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            -- ASSIGN  |PARTITIONED|
-                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
index 7e70955..977e29e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
@@ -1,92 +1,81 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$83(ASC)]  |PARTITIONED|
-              -- RANGE_PARTITION_EXCHANGE [$$83(ASC)]  |PARTITIONED|
-                -- FORWARD  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- REPLICATE  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- PRE_CLUSTERED_GROUP_BY[$$68]  |PARTITIONED|
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- STREAM_SELECT  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$68(ASC), $$69(ASC)]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$68]  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$86(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$86(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$71]  |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|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$98(ASC)]  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STABLE_SORT [$$95(ASC)]  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
-                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- ASSIGN  |PARTITIONED|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                      -- ASSIGN  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$68]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$68(ASC), $$69(ASC)]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$68]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- STREAM_SELECT  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- STABLE_SORT [$$95(ASC)]  |PARTITIONED|
+                                                    -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
-                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$71]  |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|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STABLE_SORT [$$98(ASC)]  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
+                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                 -- ASSIGN  |PARTITIONED|
-                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            -- ASSIGN  |PARTITIONED|
-                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.plan
index 8fd6f04..d41058b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.plan
@@ -1,86 +1,77 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$65(ASC)]  |PARTITIONED|
-              -- RANGE_PARTITION_EXCHANGE [$$65(ASC)]  |PARTITIONED|
-                -- FORWARD  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- REPLICATE  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- PRE_CLUSTERED_GROUP_BY[$$52]  |PARTITIONED|
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- STREAM_SELECT  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$67(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$67(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$54]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$54(ASC), $$55(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$72(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$52(ASC), $$53(ASC)]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$54]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$54(ASC), $$55(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STABLE_SORT [$$70(ASC)]  |PARTITIONED|
+                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- STABLE_SORT [$$72(ASC)]  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
                                                                     -- ASSIGN  |PARTITIONED|
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                         -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                             -- ASSIGN  |PARTITIONED|
                                                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$52]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$52(ASC), $$53(ASC)]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
-                                          -- STREAM_SELECT  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- STABLE_SORT [$$70(ASC)]  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.plan
index 8fd6f04..d41058b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.plan
@@ -1,86 +1,77 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$65(ASC)]  |PARTITIONED|
-              -- RANGE_PARTITION_EXCHANGE [$$65(ASC)]  |PARTITIONED|
-                -- FORWARD  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- REPLICATE  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- PRE_CLUSTERED_GROUP_BY[$$52]  |PARTITIONED|
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- STREAM_SELECT  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$67(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$67(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$54]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$54(ASC), $$55(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$72(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$52(ASC), $$53(ASC)]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$54]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$54(ASC), $$55(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STABLE_SORT [$$70(ASC)]  |PARTITIONED|
+                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- STABLE_SORT [$$72(ASC)]  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
                                                                     -- ASSIGN  |PARTITIONED|
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                         -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                             -- ASSIGN  |PARTITIONED|
                                                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$52]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$52(ASC), $$53(ASC)]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
-                                          -- STREAM_SELECT  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- STABLE_SORT [$$70(ASC)]  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.plan
index 9857dae..1c961a8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.plan
@@ -1,86 +1,77 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$73(ASC)]  |PARTITIONED|
-              -- RANGE_PARTITION_EXCHANGE [$$73(ASC)]  |PARTITIONED|
-                -- FORWARD  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- REPLICATE  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- PRE_CLUSTERED_GROUP_BY[$$58]  |PARTITIONED|
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- STREAM_SELECT  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$75(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$75(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$60]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$60(ASC), $$61(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$60]  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$58(ASC), $$59(ASC)]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$60]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$60(ASC), $$61(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$60]  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STABLE_SORT [$$78(ASC)]  |PARTITIONED|
+                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
                                                                     -- ASSIGN  |PARTITIONED|
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                         -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                             -- ASSIGN  |PARTITIONED|
                                                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$58]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$58(ASC), $$59(ASC)]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-                                          -- STREAM_SELECT  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- STABLE_SORT [$$78(ASC)]  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.plan
index 9857dae..1c961a8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.plan
@@ -1,86 +1,77 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$73(ASC)]  |PARTITIONED|
-              -- RANGE_PARTITION_EXCHANGE [$$73(ASC)]  |PARTITIONED|
-                -- FORWARD  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- REPLICATE  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- PRE_CLUSTERED_GROUP_BY[$$58]  |PARTITIONED|
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- STREAM_SELECT  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$75(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$75(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$60]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$60(ASC), $$61(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$60]  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$58(ASC), $$59(ASC)]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$60]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$60(ASC), $$61(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$60]  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STABLE_SORT [$$78(ASC)]  |PARTITIONED|
+                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
                                                                     -- ASSIGN  |PARTITIONED|
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                         -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                             -- ASSIGN  |PARTITIONED|
                                                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$58]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$58(ASC), $$59(ASC)]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-                                          -- STREAM_SELECT  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- STABLE_SORT [$$78(ASC)]  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.plan
index a33f232..ac9b3d4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.plan
@@ -1,22 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.ds2.ds2)  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- BTREE_SEARCH (test.ds2.ds2)  |PARTITIONED|
+                -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$40(ASC)]  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.ds2.idx)  |PARTITIONED|
-                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                        -- BTREE_SEARCH (test.ds2.idx)  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.ds1)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.ds1)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.plan
index 347599b..152d7c0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.plan
@@ -1,22 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.ds2.ds2)  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- BTREE_SEARCH (test.ds2.ds2)  |PARTITIONED|
+                -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$41(ASC)]  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.ds2.idx)  |PARTITIONED|
-                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                        -- BTREE_SEARCH (test.ds2.idx)  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.ds1)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.ds1)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/03.plan
index 04e6d71..6a72dfe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/03.plan
@@ -1,30 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$44(ASC), $$45(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$44(ASC), $$45(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- HYBRID_HASH_JOIN [$$42][$$43]  |PARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$43]  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$46(ASC), $$47(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$46(ASC), $$47(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- HYBRID_HASH_JOIN [$$44][$$45]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$44]  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$45]  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/04.plan
index 04e6d71..6a72dfe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/04.plan
@@ -1,30 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$44(ASC), $$45(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$44(ASC), $$45(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- HYBRID_HASH_JOIN [$$42][$$43]  |PARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$43]  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$46(ASC), $$47(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$46(ASC), $$47(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- HYBRID_HASH_JOIN [$$44][$$45]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$44]  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$45]  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.plan
index 97401bb..6af09d7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.plan
@@ -1,30 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$46(ASC), $$47(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$46(ASC), $$47(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|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$44]  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$48(ASC), $$49(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$48(ASC), $$49(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- HYBRID_HASH_JOIN [$$45][$$46]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$45]  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$46]  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.plan
index 4eb7b4f..5fccabc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.plan
@@ -1,31 +1,26 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$46(ASC), $$47(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$46(ASC), $$47(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.TestOpen2.TestOpen2)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TestOpen2.idx_t2_s)  |PARTITIONED|
-                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$48(ASC), $$49(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$48(ASC), $$49(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.TestOpen2.TestOpen2)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$56(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (test.TestOpen2.idx_t2_s)  |PARTITIONED|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.plan
index 6ecffbf..875f64e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.plan
@@ -1,31 +1,26 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$46(ASC), $$47(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$46(ASC), $$47(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.TestOpen2.TestOpen2)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TestOpen2.idx_t2_i64)  |PARTITIONED|
-                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$48(ASC), $$49(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$48(ASC), $$49(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.TestOpen2.TestOpen2)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$56(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (test.TestOpen2.idx_t2_i64)  |PARTITIONED|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/03.plan
index 13f9dcf..4554197 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/03.plan
@@ -1,22 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$22(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                        -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
+                                -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/04.plan
index ceb8921..2e37bd6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/04.plan
@@ -1,24 +1,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TestOpen.idx_s)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (test.TestOpen.idx_s)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/05.plan
index 2cf4812..8fced14 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/05.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/05.plan
@@ -1,24 +1,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/06.plan
index a63d87f..b5717df 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/06.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/06.plan
@@ -1,24 +1,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/07.plan
index a63d87f..b5717df 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/07.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/07.plan
@@ -1,24 +1,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/08.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/08.plan
index ef289a4..6b3856e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/08.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/08.plan
@@ -1,24 +1,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TestOpen.idx_i8)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (test.TestOpen.idx_i8)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/09.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/09.plan
index ef289a4..6b3856e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/09.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/09.plan
@@ -1,24 +1,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TestOpen.idx_i8)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (test.TestOpen.idx_i8)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/10.plan
index c3e596b..23a0c32 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/10.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/10.plan
@@ -1,24 +1,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TestOpen.idx_d)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (test.TestOpen.idx_d)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/11.plan
index 7fb98d3..7498d85 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/11.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/11.plan
@@ -1,25 +1,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$29(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$30(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$37(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
+                                  -- BTREE_SEARCH (test.TestOpen.idx_i8)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (test.TestOpen.idx_i8)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/12.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/12.plan
index fcc0784..df41877 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/12.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/12.plan
@@ -1,35 +1,32 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$29(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INTERSECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (test.TestOpen.idx_3)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (test.TestOpen.idx_4)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$30(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- INTERSECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (test.TestOpen.idx_3)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$40(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (test.TestOpen.idx_4)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |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 0aa8dc4..3920a9f 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
@@ -1,190 +1,167 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |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[$$74]  |PARTITIONED|
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- STREAM_SELECT  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$74(ASC), $$75(ASC)]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
-                                -- STREAM_PROJECT  |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|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- REPLICATE  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
-                                        -- UNION_ALL  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- STABLE_SORT [$$107(ASC)]  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx)  |PARTITIONED|
-                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                        -- STREAM_SELECT  |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|
-                                                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- NESTED_LOOP  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- DATASOURCE_SCAN (test.TweetMessages)  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_SELECT  |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|
-                                                                          -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |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[$$74]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$74(ASC), $$75(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$74]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$74(ASC), $$75(ASC)]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$95][$$74]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$95]  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- REPLICATE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
+                                      -- UNION_ALL  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STABLE_SORT [$$107(ASC)]  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx)  |PARTITIONED|
+                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- REPLICATE  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$95][$$74]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$95]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                              -- NESTED_LOOP  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
                                                     -- ASSIGN  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN (test.TweetMessages)  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- REPLICATE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                -- REPLICATE  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$74]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$74(ASC), $$75(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$95][$$74]  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$95]  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
+                                                -- UNION_ALL  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- STABLE_SORT [$$107(ASC)]  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx)  |PARTITIONED|
+                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- NESTED_LOOP  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- DATASOURCE_SCAN (test.TweetMessages)  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- REPLICATE  |PARTITIONED|
                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                     -- ASSIGN  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
-                                                  -- UNION_ALL  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- STREAM_SELECT  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- STABLE_SORT [$$107(ASC)]  |PARTITIONED|
+                                                                          -- REPLICATE  |PARTITIONED|
                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx)  |PARTITIONED|
-                                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                  -- STREAM_SELECT  |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|
-                                                                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                            -- ASSIGN  |PARTITIONED|
-                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- NESTED_LOOP  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ASSIGN  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- DATASOURCE_SCAN (test.TweetMessages)  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                              -- STREAM_SELECT  |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|
-                                                                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- ASSIGN  |PARTITIONED|
-                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_01_ps.plan
index fa450a0..df1938d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_01_ps.plan
@@ -1,50 +1,45 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-              -- RANGE_PARTITION_EXCHANGE [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-                -- FORWARD  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- REPLICATE  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- NESTED_LOOP  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |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|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- NESTED_LOOP  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- NESTED_LOOP  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- NESTED_LOOP  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_02_ps.plan
index 95adac0..d107a60 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_02_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_02_ps.plan
@@ -1,50 +1,45 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-              -- RANGE_PARTITION_EXCHANGE [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-                -- FORWARD  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- REPLICATE  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- NESTED_LOOP  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- BROADCAST_EXCHANGE  |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|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- NESTED_LOOP  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- NESTED_LOOP  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- NESTED_LOOP  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_03_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_03_ps.plan
index b7351a2..903f710 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_03_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_03_ps.plan
@@ -1,54 +1,49 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-              -- RANGE_PARTITION_EXCHANGE [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-                -- FORWARD  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- REPLICATE  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |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|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |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|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
+                              -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
+                                  -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                                            -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                                                      -- BROADCAST_EXCHANGE  |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|
+                                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |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/nested-open-index/inverted-index-join/ngram-contains_04_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_04_ps.plan
index 79aeeb4..1fcf054 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_04_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_04_ps.plan
@@ -1,54 +1,49 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-              -- RANGE_PARTITION_EXCHANGE [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-                -- FORWARD  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- REPLICATE  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |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|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index_CSX)  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |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|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
+                              -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
+                                  -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index_CSX)  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
+                                            -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index_CSX)  |PARTITIONED|
-                                                      -- BROADCAST_EXCHANGE  |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|
+                                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index_CSX)  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |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/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
index ee5703a..4a08ca9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
@@ -1,92 +1,81 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$74(ASC)]  |PARTITIONED|
-              -- RANGE_PARTITION_EXCHANGE [$$74(ASC)]  |PARTITIONED|
-                -- FORWARD  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- REPLICATE  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- PRE_CLUSTERED_GROUP_BY[$$61]  |PARTITIONED|
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- STREAM_SELECT  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$61(ASC), $$67(ASC)]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$61]  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$77(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$77(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$64]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$64(ASC), $$70(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$64]  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$89(ASC)]  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STABLE_SORT [$$86(ASC)]  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
-                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- ASSIGN  |PARTITIONED|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                      -- ASSIGN  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$61]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$61(ASC), $$67(ASC)]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$61]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- STREAM_SELECT  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- STABLE_SORT [$$86(ASC)]  |PARTITIONED|
+                                                    -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
-                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$64]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$64(ASC), $$70(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$64]  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STABLE_SORT [$$89(ASC)]  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
+                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                 -- ASSIGN  |PARTITIONED|
-                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            -- ASSIGN  |PARTITIONED|
-                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
index 7e70955..977e29e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
@@ -1,92 +1,81 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$83(ASC)]  |PARTITIONED|
-              -- RANGE_PARTITION_EXCHANGE [$$83(ASC)]  |PARTITIONED|
-                -- FORWARD  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- REPLICATE  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- PRE_CLUSTERED_GROUP_BY[$$68]  |PARTITIONED|
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- STREAM_SELECT  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$68(ASC), $$69(ASC)]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$68]  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$86(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$86(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$71]  |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|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$98(ASC)]  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STABLE_SORT [$$95(ASC)]  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
-                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- ASSIGN  |PARTITIONED|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                      -- ASSIGN  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$68]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$68(ASC), $$69(ASC)]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$68]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- STREAM_SELECT  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- STABLE_SORT [$$95(ASC)]  |PARTITIONED|
+                                                    -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
-                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$71]  |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|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STABLE_SORT [$$98(ASC)]  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
+                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                 -- ASSIGN  |PARTITIONED|
-                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            -- ASSIGN  |PARTITIONED|
-                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                  -- EMPTY_TUPLE_SOURCE  |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 e75d34e..cc2f91a 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
@@ -1,43 +1,42 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- PRE_CLUSTERED_GROUP_BY[$$71]  |PARTITIONED|
-                    {
-                      -- AGGREGATE  |LOCAL|
-                        -- MICRO_PRE_CLUSTERED_GROUP_BY[$$72]  |LOCAL|
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- STREAM_SELECT  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |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|
-                    -- STREAM_PROJECT  |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 [$$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 [$$73]  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$71]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- MICRO_PRE_CLUSTERED_GROUP_BY[$$72]  |LOCAL|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |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|
+                  -- STREAM_PROJECT  |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|
-                                  -- DATASOURCE_SCAN (tpch.LineItems)  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |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 [$$73]  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN (tpch.LineItems)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.plan
index 899f737..d7f9bcf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.plan
@@ -1,77 +1,72 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$51(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$51(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$42]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- STREAM_SELECT  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$42(ASC), $$43(ASC)]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
-                              -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$53(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$53(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$44]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- STREAM_SELECT  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$44(ASC), $$45(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$44]  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$56(ASC)]  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- ASSIGN  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$42]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- STREAM_SELECT  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$42(ASC), $$43(ASC)]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
-                                        -- STREAM_SELECT  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$44]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$44(ASC), $$45(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$44]  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                  -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STABLE_SORT [$$56(ASC)]  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ASSIGN  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.plan
index 899f737..d7f9bcf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.plan
@@ -1,77 +1,72 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$51(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$51(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$42]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- STREAM_SELECT  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$42(ASC), $$43(ASC)]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
-                              -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$53(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$53(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$44]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- STREAM_SELECT  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$44(ASC), $$45(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$44]  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$56(ASC)]  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- ASSIGN  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$42]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- STREAM_SELECT  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$42(ASC), $$43(ASC)]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
-                                        -- STREAM_SELECT  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$44]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$44(ASC), $$45(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$44]  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                  -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STABLE_SORT [$$56(ASC)]  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ASSIGN  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.plan
index ef866c9..65e8723 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.plan
@@ -1,77 +1,72 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$56(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$56(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$46]  |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]  |PARTITIONED|
-                              -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$58(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |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|
+                        -- STABLE_SORT [$$48(ASC), $$49(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- ASSIGN  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$46]  |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]  |PARTITIONED|
-                                        -- STREAM_SELECT  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$63(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |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|
+                                  -- STABLE_SORT [$$48(ASC), $$49(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                  -- STABLE_SORT [$$63(ASC)]  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ASSIGN  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.plan
index ef866c9..65e8723 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.plan
@@ -1,77 +1,72 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$56(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$56(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$46]  |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]  |PARTITIONED|
-                              -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$58(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |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|
+                        -- STABLE_SORT [$$48(ASC), $$49(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- ASSIGN  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$46]  |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]  |PARTITIONED|
-                                        -- STREAM_SELECT  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$63(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |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|
+                                  -- STABLE_SORT [$$48(ASC), $$49(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                  -- STABLE_SORT [$$63(ASC)]  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
-                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ASSIGN  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- BTREE_SEARCH (test.TweetMessages.msgCountBIx)  |PARTITIONED|
+                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |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 86e90b4..ddafba9 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
@@ -1,165 +1,152 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |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[$$62]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- STREAM_SELECT  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$62(ASC), $$63(ASC)]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$78][$$62]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |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[$$62]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- STREAM_SELECT  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$62(ASC), $$63(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$78][$$62]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
+                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                                    -- UNION_ALL  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                 -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
-                                      -- UNION_ALL  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- STREAM_SELECT  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- STABLE_SORT [$$86(ASC)]  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx)  |PARTITIONED|
-                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                  -- STREAM_SELECT  |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|
-                                                                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- ASSIGN  |PARTITIONED|
-                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- NESTED_LOOP  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (test.TweetMessages)  |PARTITIONED|
+                                                    -- STABLE_SORT [$$86(ASC)]  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_SELECT  |PARTITIONED|
+                                                        -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx)  |PARTITIONED|
+                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- REPLICATE  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- REPLICATE  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- NESTED_LOOP  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.TweetMessages)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- REPLICATE  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- REPLICATE  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$62]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$62(ASC), $$63(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$78][$$62]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                   -- REPLICATE  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                                              -- UNION_ALL  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
                                                       -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- REPLICATE  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- ASSIGN  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$62]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- STREAM_SELECT  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$62(ASC), $$63(ASC)]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
-                                        -- STREAM_PROJECT  |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|
-                                                      -- REPLICATE  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                           -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
-                                                -- UNION_ALL  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- STABLE_SORT [$$86(ASC)]  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx)  |PARTITIONED|
-                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                            -- STREAM_SELECT  |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|
-                                                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- NESTED_LOOP  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- DATASOURCE_SCAN (test.TweetMessages)  |PARTITIONED|
+                                                              -- STABLE_SORT [$$86(ASC)]  |PARTITIONED|
                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_SELECT  |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|
+                                                                  -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx)  |PARTITIONED|
+                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                      -- STREAM_SELECT  |PARTITIONED|
                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                          -- REPLICATE  |PARTITIONED|
                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                               -- ASSIGN  |PARTITIONED|
-                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- NESTED_LOOP  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (test.TweetMessages)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- REPLICATE  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- REPLICATE  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_01_ps.plan
index fa450a0..df1938d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_01_ps.plan
@@ -1,50 +1,45 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-              -- RANGE_PARTITION_EXCHANGE [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-                -- FORWARD  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- REPLICATE  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- NESTED_LOOP  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |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|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- NESTED_LOOP  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- NESTED_LOOP  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- NESTED_LOOP  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_02_ps.plan
index ef37179..29b3560a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_02_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_02_ps.plan
@@ -1,54 +1,49 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-              -- RANGE_PARTITION_EXCHANGE [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-                -- FORWARD  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- REPLICATE  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |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|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index)  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |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|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
+                              -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
+                                  -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index)  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
+                                            -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index)  |PARTITIONED|
-                                                      -- BROADCAST_EXCHANGE  |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|
+                                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index)  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |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/open-index-enforced/inverted-index-join/ngram-contains_03_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_03_ps.plan
index b7351a2..903f710 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_03_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_03_ps.plan
@@ -1,54 +1,49 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-              -- RANGE_PARTITION_EXCHANGE [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-                -- FORWARD  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- REPLICATE  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |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|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |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|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
+                              -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
+                                  -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                                            -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                                                      -- BROADCAST_EXCHANGE  |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|
+                                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |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/open-index-enforced/inverted-index-join/ngram-contains_04_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_04_ps.plan
index 79aeeb4..1fcf054 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_04_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_04_ps.plan
@@ -1,54 +1,49 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-              -- RANGE_PARTITION_EXCHANGE [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-                -- FORWARD  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- REPLICATE  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |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|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index_CSX)  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |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|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
+                              -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
+                                  -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index_CSX)  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
+                                            -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index_CSX)  |PARTITIONED|
-                                                      -- BROADCAST_EXCHANGE  |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|
+                                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index_CSX)  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |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/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
index 3066ed0..58f231f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
@@ -1,83 +1,76 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$61(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$54]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- STREAM_SELECT  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$54(ASC), $$55(ASC)]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STABLE_SORT [$$73(ASC)]  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
-                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- ASSIGN  |PARTITIONED|
-                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$54]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- STREAM_SELECT  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$64(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$64(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$57]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- STREAM_SELECT  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$57(ASC), $$58(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$57]  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$54(ASC), $$55(ASC)]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- STREAM_SELECT  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$76(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- STABLE_SORT [$$73(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
-                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                    -- ASSIGN  |PARTITIONED|
-                                                                      -- ASSIGN  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$57]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$57(ASC), $$58(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$57]  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STABLE_SORT [$$76(ASC)]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
+                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                           -- ASSIGN  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
index e183f09..4b2546a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
@@ -1,83 +1,76 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$66(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$66(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$58]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- STREAM_SELECT  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$58(ASC), $$59(ASC)]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STABLE_SORT [$$78(ASC)]  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
-                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- ASSIGN  |PARTITIONED|
-                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$58]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- STREAM_SELECT  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$69(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$69(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$61]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- STREAM_SELECT  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$61(ASC), $$62(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$61]  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$58(ASC), $$59(ASC)]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- STREAM_SELECT  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$81(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- STABLE_SORT [$$78(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
-                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                    -- ASSIGN  |PARTITIONED|
-                                                                      -- ASSIGN  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$61]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$61(ASC), $$62(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$61]  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STABLE_SORT [$$81(ASC)]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
+                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                           -- ASSIGN  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-03.plan
index b266aa8..093ddc0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-03.plan
@@ -1,17 +1,16 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.TestOpen.idx_xyz)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.TestOpen.idx_xyz)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-04.plan
index f88176b..741828de 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-04.plan
@@ -1,17 +1,16 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.TestOpen.idx_xyz)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.TestOpen.idx_xyz)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05.plan
index ea60aab..0732795 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05.plan
@@ -1,26 +1,23 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$36(ASC), $$37(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$36(ASC), $$37(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- HYBRID_HASH_JOIN [$$34][$$35]  |PARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$38(ASC), $$39(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- HYBRID_HASH_JOIN [$$36][$$37]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05_ps.plan
index 45133f4..eb4e601 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05_ps.plan
@@ -1,56 +1,51 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$36(ASC), $$37(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$36(ASC), $$37(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 [$$34][$$35]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$38(ASC), $$39(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 [$$36][$$37]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$34][$$35]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$36][$$37]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06.plan
index ea60aab..0732795 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06.plan
@@ -1,26 +1,23 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$36(ASC), $$37(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$36(ASC), $$37(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- HYBRID_HASH_JOIN [$$34][$$35]  |PARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$38(ASC), $$39(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- HYBRID_HASH_JOIN [$$36][$$37]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06_ps.plan
index 45133f4..eb4e601 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06_ps.plan
@@ -1,56 +1,51 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$36(ASC), $$37(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$36(ASC), $$37(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 [$$34][$$35]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$38(ASC), $$39(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 [$$36][$$37]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$34][$$35]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$36][$$37]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07.plan
index d347e73..09a9501 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07.plan
@@ -1,26 +1,23 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$38(ASC), $$39(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- HYBRID_HASH_JOIN [$$35][$$36]  |PARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$40(ASC), $$41(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- HYBRID_HASH_JOIN [$$37][$$38]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.plan
index 7dfa995..df1a86f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.plan
@@ -1,56 +1,51 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$38(ASC), $$39(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 [$$35][$$36]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$40(ASC), $$41(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 [$$37][$$38]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$35][$$36]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$37][$$38]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08.plan
index e2f07be..e7a7e6c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08.plan
@@ -1,27 +1,24 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$38(ASC), $$39(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$40(ASC), $$41(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (test.TestOpen2.TestOpen2)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (test.TestOpen2.TestOpen2)  |PARTITIONED|
+                        -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (test.TestOpen2.idx_t2_s)  |PARTITIONED|
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.TestOpen2.idx_t2_s)  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08_ps.plan
index af04b9c..bd8b971 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08_ps.plan
@@ -1,58 +1,53 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.TestOpen2.TestOpen2)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (test.TestOpen2.TestOpen2)  |PARTITIONED|
+                                -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (test.TestOpen2.idx_t2_s)  |PARTITIONED|
-                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TestOpen2.TestOpen2)  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.TestOpen2.idx_t2_s)  |PARTITIONED|
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.TestOpen2.TestOpen2)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (test.TestOpen2.idx_t2_s)  |PARTITIONED|
-                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- BTREE_SEARCH (test.TestOpen2.idx_t2_s)  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09.plan
index 7afc23e..afc9224 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09.plan
@@ -1,27 +1,24 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$38(ASC), $$39(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$40(ASC), $$41(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (test.TestOpen2.TestOpen2)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (test.TestOpen2.TestOpen2)  |PARTITIONED|
+                        -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (test.TestOpen2.idx_t2_i64)  |PARTITIONED|
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.TestOpen2.idx_t2_i64)  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09_ps.plan
index aaa92c7..a98c6a7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09_ps.plan
@@ -1,58 +1,53 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$38(ASC), $$39(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.TestOpen2.TestOpen2)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (test.TestOpen2.TestOpen2)  |PARTITIONED|
+                                -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (test.TestOpen2.idx_t2_i64)  |PARTITIONED|
-                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TestOpen2.TestOpen2)  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.TestOpen2.idx_t2_i64)  |PARTITIONED|
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.TestOpen2.TestOpen2)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (test.TestOpen2.idx_t2_i64)  |PARTITIONED|
-                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- BTREE_SEARCH (test.TestOpen2.idx_t2_i64)  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04.plan
index c32b198..1fd5284 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04.plan
@@ -1,22 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                        -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
+                                -- BTREE_SEARCH (test.TestOpen.idx_s)  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (test.TestOpen.idx_s)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.plan
index bc44f0e..bfa6300 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.plan
@@ -1,48 +1,45 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$20(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$21(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                                -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (test.TestOpen.idx_s)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.TestOpen.idx_s)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                -- BTREE_SEARCH (test.TestOpen.idx_s)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (test.TestOpen.idx_s)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05.plan
index bb301a0..57a6050 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05.plan
@@ -1,22 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                        -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
+                                -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.plan
index bb1fe46..308a979 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.plan
@@ -1,48 +1,45 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$20(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$21(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                                -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06.plan
index b5d7af2..156ff5a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06.plan
@@ -1,22 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
+                                -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.plan
index 5181114..78ac4aa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.plan
@@ -1,48 +1,45 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$20(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$21(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                                -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07.plan
index b5d7af2..156ff5a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07.plan
@@ -1,22 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
+                                -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.plan
index 5181114..78ac4aa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.plan
@@ -1,48 +1,45 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$20(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$21(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                                -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (test.TestOpen.idx_i64)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08.plan
index cc01990..71142c3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08.plan
@@ -1,22 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
+                                -- BTREE_SEARCH (test.TestOpen.idx_i8)  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (test.TestOpen.idx_i8)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.plan
index fca170c..70ec934 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.plan
@@ -1,48 +1,45 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$20(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$21(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                                -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (test.TestOpen.idx_i8)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.TestOpen.idx_i8)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                -- BTREE_SEARCH (test.TestOpen.idx_i8)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (test.TestOpen.idx_i8)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09.plan
index cc01990..71142c3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09.plan
@@ -1,22 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
+                                -- BTREE_SEARCH (test.TestOpen.idx_i8)  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (test.TestOpen.idx_i8)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.plan
index fca170c..70ec934 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.plan
@@ -1,48 +1,45 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$20(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$21(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                                -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (test.TestOpen.idx_i8)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.TestOpen.idx_i8)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                -- BTREE_SEARCH (test.TestOpen.idx_i8)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (test.TestOpen.idx_i8)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10.plan
index 71a312f..c830e00 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10.plan
@@ -1,22 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
+                                -- BTREE_SEARCH (test.TestOpen.idx_d)  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (test.TestOpen.idx_d)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105.plan
index a769c18..96aaed6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105.plan
@@ -1,33 +1,31 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                        -- INTERSECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- INTERSECT  |PARTITIONED|
+                            -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.TestOpen.idx_3)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TestOpen.idx_3)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.TestOpen.idx_4)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TestOpen.idx_4)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.plan
index 4f87b34..b5eca38 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.plan
@@ -1,70 +1,67 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$24(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$25(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                                -- INTERSECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INTERSECT  |PARTITIONED|
+                                    -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
+                                            -- BTREE_SEARCH (test.TestOpen.idx_3)  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- BTREE_SEARCH (test.TestOpen.idx_3)  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
+                                            -- BTREE_SEARCH (test.TestOpen.idx_4)  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- BTREE_SEARCH (test.TestOpen.idx_4)  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                                        -- INTERSECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- INTERSECT  |PARTITIONED|
+                                            -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- BTREE_SEARCH (test.TestOpen.idx_3)  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- BTREE_SEARCH (test.TestOpen.idx_3)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- BTREE_SEARCH (test.TestOpen.idx_4)  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- BTREE_SEARCH (test.TestOpen.idx_4)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.plan
index 62bf5b4..0b6b1c4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.plan
@@ -1,48 +1,45 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$20(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$21(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                                -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (test.TestOpen.idx_d)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.TestOpen.idx_d)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                -- BTREE_SEARCH (test.TestOpen.idx_d)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (test.TestOpen.idx_d)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11.plan
index d408e8f..29b0374 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11.plan
@@ -1,23 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (test.TestOpen.idx_i8)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.TestOpen.idx_i8)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.plan
index dbe5b68..bc5e8ab 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.plan
@@ -1,50 +1,45 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$23(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- BTREE_SEARCH (test.TestOpen.idx_i8)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- BTREE_SEARCH (test.TestOpen.idx_i8)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$24(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.TestOpen.idx_i8)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH (test.TestOpen.idx_i8)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orderby-desc-using-gby_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orderby-desc-using-gby_ps.plan
index cb25666..cc0b3e9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orderby-desc-using-gby_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orderby-desc-using-gby_ps.plan
@@ -1,44 +1,41 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$name(DESC), $$age(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$name(DESC), $$age(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- SORT_GROUP_BY[$$47, $$48]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- HASH_PARTITION_EXCHANGE [$$47, $$48]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$name(DESC), $$age(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$name(DESC), $$age(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- SORT_GROUP_BY[$$51, $$52]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- HASH_PARTITION_EXCHANGE [$$51, $$52]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (`gby-using-orderby-desc`.Customers)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- SORT_GROUP_BY[$$47, $$48]  |PARTITIONED|
-                                      {
-                                        -- AGGREGATE  |LOCAL|
-                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                      }
-                                -- HASH_PARTITION_EXCHANGE [$$47, $$48]  |PARTITIONED|
+                              -- DATASOURCE_SCAN (`gby-using-orderby-desc`.Customers)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- SORT_GROUP_BY[$$51, $$52]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- HASH_PARTITION_EXCHANGE [$$51, $$52]  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (`gby-using-orderby-desc`.Customers)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (`gby-using-orderby-desc`.Customers)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |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 6bbed20..c7db207 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
@@ -13,22 +13,20 @@
                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
+                      -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01_ps.plan
index 9c39aef..a568a2a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01_ps.plan
@@ -13,48 +13,45 @@
                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$24(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$24(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
+                              -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                      -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
+                                        -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |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 4c73e76..414c78b 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
@@ -13,22 +13,20 @@
                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$26(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$26(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
+                      -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02_ps.plan
index f79649b..9dffd76 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02_ps.plan
@@ -13,48 +13,45 @@
                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$26(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$26(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
+                              -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
+                                        -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |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 6bbed20..c7db207 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
@@ -13,22 +13,20 @@
                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
+                      -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01_ps.plan
index 9c39aef..a568a2a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01_ps.plan
@@ -13,48 +13,45 @@
                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$24(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$24(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
+                              -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                      -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
+                                        -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |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 4c73e76..414c78b 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
@@ -13,22 +13,20 @@
                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$26(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$26(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
+                      -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02_ps.plan
index f79649b..9dffd76 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02_ps.plan
@@ -13,48 +13,45 @@
                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$26(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$26(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
+                              -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
+                                        -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/p_sort_join/p_sort_join.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/p_sort_join/p_sort_join.plan
index 0ea33f1..8261d45 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/p_sort_join/p_sort_join.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/p_sort_join/p_sort_join.plan
@@ -1,76 +1,71 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$id2(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$id2(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- SORT_GROUP_BY[$$87]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- HYBRID_HASH_JOIN [$$87][$$90]  |PARTITIONED|
-                                -- HASH_PARTITION_EXCHANGE [$$87]  |PARTITIONED|
-                                  -- STABLE_SORT [$$87(ASC)]  |PARTITIONED|
-                                    -- RANGE_PARTITION_EXCHANGE [$$87(ASC)] RANGE_MAP:{SPLIT:1}  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$id2(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$id2(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- SORT_GROUP_BY[$$93]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$93][$$96]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$93]  |PARTITIONED|
+                                -- STABLE_SORT [$$93(ASC)]  |PARTITIONED|
+                                  -- RANGE_PARTITION_EXCHANGE [$$93(ASC)] RANGE_MAP:{SPLIT:1}  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.TestDS1)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.TestDS1)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- HASH_PARTITION_EXCHANGE [$$90]  |PARTITIONED|
-                                  -- STABLE_SORT [$$90(ASC)]  |PARTITIONED|
-                                    -- RANGE_PARTITION_EXCHANGE [$$90(ASC)] RANGE_MAP:{SPLIT:1}  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$96]  |PARTITIONED|
+                                -- STABLE_SORT [$$96(ASC)]  |PARTITIONED|
+                                  -- RANGE_PARTITION_EXCHANGE [$$96(ASC)] RANGE_MAP:{SPLIT:1}  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.TestDS2)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.TestDS2)  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- SORT_GROUP_BY[$$93]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$93][$$96]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$93]  |PARTITIONED|
+                                        -- STABLE_SORT [$$93(ASC)]  |PARTITIONED|
+                                          -- RANGE_PARTITION_EXCHANGE [$$93(ASC)] RANGE_MAP:{SPLIT:1}  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- SORT_GROUP_BY[$$87]  |PARTITIONED|
-                                      {
-                                        -- AGGREGATE  |LOCAL|
-                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                      }
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- HYBRID_HASH_JOIN [$$87][$$90]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$87]  |PARTITIONED|
-                                          -- STABLE_SORT [$$87(ASC)]  |PARTITIONED|
-                                            -- RANGE_PARTITION_EXCHANGE [$$87(ASC)] RANGE_MAP:{SPLIT:1}  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (test.TestDS1)  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (test.TestDS1)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$90]  |PARTITIONED|
-                                          -- STABLE_SORT [$$90(ASC)]  |PARTITIONED|
-                                            -- RANGE_PARTITION_EXCHANGE [$$90(ASC)] RANGE_MAP:{SPLIT:1}  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$96]  |PARTITIONED|
+                                        -- STABLE_SORT [$$96(ASC)]  |PARTITIONED|
+                                          -- RANGE_PARTITION_EXCHANGE [$$96(ASC)] RANGE_MAP:{SPLIT:1}  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
                                               -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (test.TestDS2)  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (test.TestDS2)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/p_sort_seq_merge/p_sort_seq_merge.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/p_sort_seq_merge/p_sort_seq_merge.plan
index c91f5f6..a272cca 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/p_sort_seq_merge/p_sort_seq_merge.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/p_sort_seq_merge/p_sort_seq_merge.plan
@@ -1,34 +1,32 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- SEQUENTIAL_MERGE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- SEQUENTIAL_MERGE_EXCHANGE  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$20(ASC), $$19(ASC)]  |PARTITIONED|
-                      -- RANGE_PARTITION_EXCHANGE [$$20(ASC), $$19(ASC)]  |PARTITIONED|
-                        -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$21(ASC), $$20(ASC)]  |PARTITIONED|
+                  -- RANGE_PARTITION_EXCHANGE [$$21(ASC), $$20(ASC)]  |PARTITIONED|
+                    -- FORWARD  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
+                                -- DATASOURCE_SCAN (test.TestDS)  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (test.TestDS)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- AGGREGATE  |UNPARTITIONED|
-                              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                -- AGGREGATE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |UNPARTITIONED|
+                          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                            -- AGGREGATE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- REPLICATE  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- REPLICATE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.TestDS)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.TestDS)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/insert-and-scan-primary-key-index-with-secondary.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/insert-and-scan-primary-key-index-with-secondary.plan
index 04583cd..512f7a8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/insert-and-scan-primary-key-index-with-secondary.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/insert-and-scan-primary-key-index-with-secondary.plan
@@ -8,12 +8,11 @@
               -- INSERT_DELETE  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- MATERIALIZE  |PARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$16]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$17]  |PARTITIONED|
                       -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.myData)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |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/primary-key-index/insert-primary-key-index-with-auto-gen-pk.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/insert-primary-key-index-with-auto-gen-pk.plan
index 5f89a6d..32d087c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/insert-primary-key-index-with-auto-gen-pk.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/insert-primary-key-index-with-auto-gen-pk.plan
@@ -5,6 +5,5 @@
         -- HASH_PARTITION_EXCHANGE [$$3]  |PARTITIONED|
           -- ASSIGN  |UNPARTITIONED|
             -- ASSIGN  |UNPARTITIONED|
-              -- STREAM_PROJECT  |UNPARTITIONED|
-                -- ASSIGN  |UNPARTITIONED|
-                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+              -- ASSIGN  |UNPARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/load-primary-key-index-with-secondary.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/load-primary-key-index-with-secondary.plan
index 689fae14..bde8db0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/load-primary-key-index-with-secondary.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/load-primary-key-index-with-secondary.plan
@@ -48,20 +48,19 @@
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- STABLE_SORT [$$5(ASC), $$6(ASC), $$2(ASC)]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- REPLICATE  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BULKLOAD  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$2(ASC)]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$2]  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (loadable_dv.loadable_ds)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- REPLICATE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BULKLOAD  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$2(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$2]  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (loadable_dv.loadable_ds)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/upsert-primary-key-index-with-secondary.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/upsert-primary-key-index-with-secondary.plan
index 9ad7e30..beabd3d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/upsert-primary-key-index-with-secondary.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/upsert-primary-key-index-with-secondary.plan
@@ -5,12 +5,11 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- INDEX_INSERT_DELETE  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INSERT_DELETE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$3]  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- INSERT_DELETE  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$3]  |PARTITIONED|
+                        -- ASSIGN  |UNPARTITIONED|
                           -- ASSIGN  |UNPARTITIONED|
-                            -- ASSIGN  |UNPARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/pull_select_above_eq_join.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/pull_select_above_eq_join.plan
index f7dc3b0..9c1996b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/pull_select_above_eq_join.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/pull_select_above_eq_join.plan
@@ -1,23 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- HYBRID_HASH_JOIN [$$38][$$39]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (`pull-select-above-eq-join`.Users)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (`pull-select-above-eq-join`.Visitors)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$40][$$41]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$40]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (`pull-select-above-eq-join`.Users)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$41]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (`pull-select-above-eq-join`.Visitors)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/push-project-through-group.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/push-project-through-group.plan
index ee985d4..d8108fb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/push-project-through-group.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/push-project-through-group.plan
@@ -1,32 +1,29 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$42]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- STREAM_SELECT  |LOCAL|
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$45][$$44]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$45]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$45]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- STREAM_SELECT  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$45]  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$48][$$47]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (fuzzyjoin_080.DBLP)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (fuzzyjoin_080.DBLP)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$44]  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$47]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (fuzzyjoin_080.DBLP)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (fuzzyjoin_080.DBLP)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/q01_pricing_summary_report_nt_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q01_pricing_summary_report_nt_ps.plan
index b5c8603..b1c772d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/q01_pricing_summary_report_nt_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q01_pricing_summary_report_nt_ps.plan
@@ -13,62 +13,59 @@
                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EXTERNAL_GROUP_BY[$$217, $$218]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- HASH_PARTITION_EXCHANGE [$$217, $$218]  |PARTITIONED|
-                          -- EXTERNAL_GROUP_BY[$$192, $$193]  |PARTITIONED|
-                                  {
-                                    -- AGGREGATE  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                  }
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EXTERNAL_GROUP_BY[$$217, $$218]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- HASH_PARTITION_EXCHANGE [$$217, $$218]  |PARTITIONED|
+                        -- EXTERNAL_GROUP_BY[$$192, $$193]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EXTERNAL_GROUP_BY[$$217, $$218]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- HASH_PARTITION_EXCHANGE [$$217, $$218]  |PARTITIONED|
-                                    -- EXTERNAL_GROUP_BY[$$192, $$193]  |PARTITIONED|
-                                            {
-                                              -- AGGREGATE  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                            }
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- EXTERNAL_GROUP_BY[$$217, $$218]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- HASH_PARTITION_EXCHANGE [$$217, $$218]  |PARTITIONED|
+                                  -- EXTERNAL_GROUP_BY[$$192, $$193]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
                                           -- ASSIGN  |PARTITIONED|
-                                            -- STREAM_SELECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |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 156c096..42e72a1 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
@@ -1,78 +1,70 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$205][$$233]  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- HYBRID_HASH_JOIN [$$205][$$233]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (q08_group_by.Supplier)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (q08_group_by.Supplier)  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$233]  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$230][$$212]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$230]  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$233]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- HYBRID_HASH_JOIN [$$230][$$212]  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$230]  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$206][$$208]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$206]  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (q08_group_by.LineItem)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$208]  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$206][$$208]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$206]  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- HYBRID_HASH_JOIN [$$221][$$209]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$221]  |PARTITIONED|
-                                        -- STREAM_SELECT  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN (q08_group_by.Orders)  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$209]  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |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 [$$210]  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |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|
-                                                      -- 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|
-                      -- HASH_PARTITION_EXCHANGE [$$212]  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- STREAM_SELECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (q08_group_by.Part)  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (q08_group_by.LineItem)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$208]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$221][$$209]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$221]  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN (q08_group_by.Orders)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$209]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$218][$$210]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$218]  |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 [$$210]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- HYBRID_HASH_JOIN [$$216][$$211]  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$216]  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (q08_group_by.Nation)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$211]  |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|
+                    -- HASH_PARTITION_EXCHANGE [$$212]  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (q08_group_by.Part)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |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 fde2a09..9c30f8f 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
@@ -1,55 +1,49 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$147][$$169]  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$147]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- HYBRID_HASH_JOIN [$$147][$$169]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$147]  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (q09_group_by.Part)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (q09_group_by.Part)  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$169]  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$148, $$149][$$169, $$154]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$148, $$149]  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$169]  |PARTITIONED|
-                -- STREAM_PROJECT  |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|
+                          -- DATASOURCE_SCAN (q09_group_by.Partsupp)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (q09_group_by.Partsupp)  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$169, $$154]  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$150][$$154]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$150]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$169, $$154]  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$150][$$154]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$150]  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$157][$$151]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$157]  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (q09_group_by.Supplier)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$151]  |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 [$$154]  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |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|
-                                      -- 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 [$$154]  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (q09_group_by.LineItem)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (q09_group_by.LineItem)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/quantifiers/query-ASTERIXDB-2696.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/quantifiers/query-ASTERIXDB-2696.plan
index 26b18be..4da9a4a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/quantifiers/query-ASTERIXDB-2696.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/quantifiers/query-ASTERIXDB-2696.plan
@@ -1,25 +1,23 @@
 -- DISTRIBUTE_RESULT  |LOCAL|
   -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-    -- STREAM_PROJECT  |LOCAL|
-      -- ASSIGN  |LOCAL|
-        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-          -- PRE_CLUSTERED_GROUP_BY[$$67]  |LOCAL|
-                  {
+    -- ASSIGN  |LOCAL|
+      -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+        -- PRE_CLUSTERED_GROUP_BY[$$75]  |LOCAL|
+                {
+                  -- AGGREGATE  |LOCAL|
                     -- AGGREGATE  |LOCAL|
-                      -- AGGREGATE  |LOCAL|
-                        -- SUBPLAN  |LOCAL|
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- STREAM_SELECT  |LOCAL|
-                                      -- UNNEST  |LOCAL|
-                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-              -- STABLE_SORT [$$67(ASC)]  |LOCAL|
-                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                  -- STREAM_PROJECT  |UNPARTITIONED|
-                    -- ASSIGN  |UNPARTITIONED|
-                      -- UNNEST  |UNPARTITIONED|
-                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
\ No newline at end of file
+                      -- SUBPLAN  |LOCAL|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- UNNEST  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+            -- STABLE_SORT [$$75(ASC)]  |LOCAL|
+              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                -- ASSIGN  |UNPARTITIONED|
+                  -- UNNEST  |UNPARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
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 6435a29..cff15c6 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
@@ -1,50 +1,38 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     -- STREAM_LIMIT  |UNPARTITIONED|
-      -- STREAM_PROJECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$$47(DESC), $$48(DESC) ]  |PARTITIONED|
-            -- STREAM_LIMIT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |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 [$$47][$$56]  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$47]  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$47(DESC), $$48(DESC) ]  |PARTITIONED|
+          -- STREAM_LIMIT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |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 [$$47][$$56]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$47]  |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|
-                          -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- STREAM_SELECT  |PARTITIONED|
-                                -- SUBPLAN  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- UNNEST  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- SUBPLAN  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- UNNEST  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- NESTED_LOOP  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- NESTED_LOOP  |PARTITIONED|
+                                    -- ASSIGN  |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|
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- REPLICATE  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- STREAM_PROJECT  |PARTITIONED|
@@ -52,3 +40,11 @@
                                                 -- DATASOURCE_SCAN (TinySocial.FacebookUsers)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- BROADCAST_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|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1671.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1671.plan
index 83b412d..c0b4e80 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1671.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1671.plan
@@ -1,17 +1,16 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     -- STREAM_LIMIT  |UNPARTITIONED|
-      -- STREAM_PROJECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- SORT_MERGE_EXCHANGE [$$19(ASC), $$20(ASC) ]  |PARTITIONED|
-              -- STREAM_LIMIT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [topK: 0] [$$19(ASC), $$20(ASC)]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$20(ASC), $$21(ASC) ]  |PARTITIONED|
+            -- STREAM_LIMIT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [topK: 0] [$$20(ASC), $$21(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan
index 1feedcd..63b4f0f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan
@@ -1,25 +1,23 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$159, $$160]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$159, $$160]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$133, $$134]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC) ]  |PARTITIONED|
+        -- SORT_GROUP_BY[$$163, $$164]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$163, $$164]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$137, $$138]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806_ps.plan
index 0d669e0..640eed5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806_ps.plan
@@ -1,59 +1,56 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- SORT_GROUP_BY[$$159, $$160]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- HASH_PARTITION_EXCHANGE [$$159, $$160]  |PARTITIONED|
-                          -- SORT_GROUP_BY[$$133, $$134]  |PARTITIONED|
-                                  {
-                                    -- AGGREGATE  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                  }
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- SORT_GROUP_BY[$$163, $$164]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- HASH_PARTITION_EXCHANGE [$$163, $$164]  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$137, $$138]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- SORT_GROUP_BY[$$159, $$160]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- HASH_PARTITION_EXCHANGE [$$159, $$160]  |PARTITIONED|
-                                    -- SORT_GROUP_BY[$$133, $$134]  |PARTITIONED|
-                                            {
-                                              -- AGGREGATE  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                            }
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- STREAM_SELECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- SORT_GROUP_BY[$$163, $$164]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- HASH_PARTITION_EXCHANGE [$$163, $$164]  |PARTITIONED|
+                                  -- SORT_GROUP_BY[$$137, $$138]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2354.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2354.plan
index 93b59d4..6bdc847 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2354.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2354.plan
@@ -1,45 +1,42 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$114(ASC), $$115(ASC), $$116(ASC) ]  |PARTITIONED|
-        -- STABLE_SORT [$$114(ASC), $$115(ASC), $$116(ASC)]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$127(ASC), $$128(ASC), $$129(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$127(ASC), $$128(ASC), $$129(ASC)]  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- UNION_ALL  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
                       -- ASSIGN  |PARTITIONED|
                         -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- REPLICATE  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_SELECT  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (Metadata.Dataset)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (Metadata.Dataset)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- NESTED_LOOP  |PARTITIONED|
-                        -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                          -- STREAM_SELECT  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (Metadata.Index)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- NESTED_LOOP  |PARTITIONED|
+                      -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_SELECT  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (Metadata.Dataset)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (Metadata.Index)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (Metadata.Dataset)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2354_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2354_ps.plan
index 323e1cd..32a3a17 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2354_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2354_ps.plan
@@ -2,8 +2,8 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STABLE_SORT [$$114(ASC), $$115(ASC), $$116(ASC)]  |PARTITIONED|
-          -- RANGE_PARTITION_EXCHANGE [$$114(ASC), $$115(ASC), $$116(ASC)]  |PARTITIONED|
+        -- STABLE_SORT [$$127(ASC), $$128(ASC), $$129(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$127(ASC), $$128(ASC), $$129(ASC)]  |PARTITIONED|
             -- FORWARD  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- REPLICATE  |PARTITIONED|
@@ -12,41 +12,38 @@
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
                               -- ASSIGN  |PARTITIONED|
                                 -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- REPLICATE  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_SELECT  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (Metadata.Dataset)  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- DATASOURCE_SCAN (Metadata.Dataset)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- NESTED_LOOP  |PARTITIONED|
-                                -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (Metadata.Index)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                  -- REPLICATE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- NESTED_LOOP  |PARTITIONED|
+                              -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_SELECT  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (Metadata.Dataset)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (Metadata.Index)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- REPLICATE  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (Metadata.Dataset)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
               -- BROADCAST_EXCHANGE  |PARTITIONED|
                 -- AGGREGATE  |UNPARTITIONED|
                   -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
@@ -59,38 +56,35 @@
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
                                     -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
                                         -- ASSIGN  |PARTITIONED|
                                           -- ASSIGN  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- REPLICATE  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- REPLICATE  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN (Metadata.Dataset)  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- DATASOURCE_SCAN (Metadata.Dataset)  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- NESTED_LOOP  |PARTITIONED|
-                                          -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_SELECT  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- DATASOURCE_SCAN (Metadata.Index)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- NESTED_LOOP  |PARTITIONED|
+                                        -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_SELECT  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (Metadata.Dataset)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (Metadata.Index)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- REPLICATE  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (Metadata.Dataset)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |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 0993f9b..0168b4f 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
@@ -1,36 +1,33 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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[$$46]  |PARTITIONED|
-                            {
-                              -- AGGREGATE  |LOCAL|
-                                -- STREAM_SELECT  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                            }
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- HYBRID_HASH_JOIN [$$46][$$49]  |PARTITIONED|
-                                  -- HASH_PARTITION_EXCHANGE [$$46]  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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[$$46]  |PARTITIONED|
+                          {
+                            -- AGGREGATE  |LOCAL|
+                              -- STREAM_SELECT  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                          }
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$46][$$49]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$46]  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.customers)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |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|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$49]  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |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 f2e47fa..d63b567 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
@@ -1,76 +1,71 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |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[$$46]  |PARTITIONED|
-                                    {
-                                      -- AGGREGATE  |LOCAL|
-                                        -- STREAM_SELECT  |LOCAL|
-                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                    }
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- HYBRID_HASH_JOIN [$$46][$$49]  |PARTITIONED|
-                                          -- HASH_PARTITION_EXCHANGE [$$46]  |PARTITIONED|
-                                            -- STREAM_PROJECT  |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 [$$49]  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |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[$$46]  |PARTITIONED|
+                                  {
+                                    -- AGGREGATE  |LOCAL|
+                                      -- STREAM_SELECT  |LOCAL|
+                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                  }
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
+                              -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- PRE_CLUSTERED_GROUP_BY[$$46]  |PARTITIONED|
-                                              {
-                                                -- AGGREGATE  |LOCAL|
-                                                  -- STREAM_SELECT  |LOCAL|
-                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                              }
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
+                                      -- HYBRID_HASH_JOIN [$$46][$$49]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$46]  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.customers)  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- HYBRID_HASH_JOIN [$$46][$$49]  |PARTITIONED|
-                                                    -- HASH_PARTITION_EXCHANGE [$$46]  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$49]  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |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[$$46]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- STREAM_SELECT  |LOCAL|
+                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- HYBRID_HASH_JOIN [$$46][$$49]  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$46]  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN (test.customers)  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |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|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$49]  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2700.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2700.plan
index 4587b19..cc46216 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2700.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2700.plan
@@ -1,35 +1,32 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (bigfun.GleambookMessagesComposite.GleambookMessagesComposite)  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- BTREE_SEARCH (bigfun.GleambookMessagesComposite.GleambookMessagesComposite)  |PARTITIONED|
+                -- STABLE_SORT [$$62(ASC), $$63(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$59(ASC), $$60(ASC)]  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (bigfun.GleambookMessagesComposite.authorIdIx)  |PARTITIONED|
-                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- SORT_MERGE_EXCHANGE [$$45(ASC) ]  |PARTITIONED|
-                                    -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (bigfun.GleambookMessagesComposite.authorIdIx)  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- SORT_MERGE_EXCHANGE [$$48(ASC) ]  |PARTITIONED|
+                                -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
-                                          -- STREAM_SELECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- BTREE_SEARCH (bigfun.GleambookUsersComposite.GleambookUsersComposite)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STABLE_SORT [$$56(ASC), $$57(ASC)]  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- BTREE_SEARCH (bigfun.GleambookUsersComposite.usrSinceIx)  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- ASSIGN  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH (bigfun.GleambookUsersComposite.GleambookUsersComposite)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STABLE_SORT [$$59(ASC), $$60(ASC)]  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH (bigfun.GleambookUsersComposite.usrSinceIx)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-3334.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-3334.plan
index bbb94e3..54dec5e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-3334.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-3334.plan
@@ -1,61 +1,56 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- SORT_GROUP_BY[$$1112]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$1112]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$1113]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$1113, $$1114][$$Employee Name, $$Call Center Region]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$1114, $$1113]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- ASSIGN  |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 (test.collection0)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- SORT_GROUP_BY[$$1120]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$1120]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$1121]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$1121, $$1122][$$Employee Name, $$Call Center Region]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$1122, $$1121]  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- SORT_GROUP_BY[$$1109, $$1110]  |PARTITIONED|
+                                -- REPLICATE  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- SORT_GROUP_BY[$$1117, $$1118]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- HASH_PARTITION_EXCHANGE [$$1117, $$1118]  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$1125, $$1124]  |PARTITIONED|
                                         {
                                           -- AGGREGATE  |LOCAL|
                                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                                         }
-                                  -- HASH_PARTITION_EXCHANGE [$$1109, $$1110]  |PARTITIONED|
-                                    -- SORT_GROUP_BY[$$1117, $$1116]  |PARTITIONED|
-                                            {
-                                              -- AGGREGATE  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                            }
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- STREAM_SELECT  |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 (test.collection0)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_SELECT  |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 (test.collection0)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-3512.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-3512.plan
index 9160a43..5c94534 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-3512.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-3512.plan
@@ -2,27 +2,25 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- UNION_ALL  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_SELECT  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_SELECT  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.collection2.collection2)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$117(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.collection2.idx1)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.collection2.collection2)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$123(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.collection2.idx1)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-2.plan
index 70e5fb0..044420e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-2.plan
@@ -1,37 +1,36 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$120, $$121]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$120(ASC), $$121(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$120, $$121]  |PARTITIONED|
-                  -- PRE_CLUSTERED_GROUP_BY[$$106, $$107]  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$106(ASC), $$107(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC) ]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$128, $$129]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$128(ASC), $$129(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$128, $$129]  |PARTITIONED|
+                -- PRE_CLUSTERED_GROUP_BY[$$114, $$115]  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$114(ASC), $$115(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-2_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-2_ps.plan
index 74741ef..bbb0517 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-2_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-2_ps.plan
@@ -1,83 +1,82 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$120, $$121]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$128, $$129]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$128(ASC), $$129(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$128, $$129]  |PARTITIONED|
+                            -- PRE_CLUSTERED_GROUP_BY[$$114, $$115]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$114(ASC), $$115(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$120(ASC), $$121(ASC)]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$120, $$121]  |PARTITIONED|
-                              -- PRE_CLUSTERED_GROUP_BY[$$106, $$107]  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$128, $$129]  |PARTITIONED|
                                       {
                                         -- AGGREGATE  |LOCAL|
-                                          -- STREAM_SELECT  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
                                       }
                                       {
                                         -- AGGREGATE  |LOCAL|
-                                          -- STREAM_SELECT  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
                                       }
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$106(ASC), $$107(ASC)]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$120, $$121]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$120(ASC), $$121(ASC)]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$120, $$121]  |PARTITIONED|
-                                        -- PRE_CLUSTERED_GROUP_BY[$$106, $$107]  |PARTITIONED|
-                                                {
-                                                  -- AGGREGATE  |LOCAL|
-                                                    -- STREAM_SELECT  |LOCAL|
-                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                }
-                                                {
-                                                  -- AGGREGATE  |LOCAL|
-                                                    -- STREAM_SELECT  |LOCAL|
-                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                }
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$106(ASC), $$107(ASC)]  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- STABLE_SORT [$$128(ASC), $$129(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$128, $$129]  |PARTITIONED|
+                                      -- PRE_CLUSTERED_GROUP_BY[$$114, $$115]  |PARTITIONED|
+                                              {
+                                                -- AGGREGATE  |LOCAL|
+                                                  -- STREAM_SELECT  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                              }
+                                              {
+                                                -- AGGREGATE  |LOCAL|
+                                                  -- STREAM_SELECT  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                              }
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STABLE_SORT [$$114(ASC), $$115(ASC)]  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-3.plan
index 25952ed..fdbbcce 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-3.plan
@@ -1,37 +1,36 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$130, $$131]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$130(ASC), $$131(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$130, $$131]  |PARTITIONED|
-                  -- PRE_CLUSTERED_GROUP_BY[$$117, $$118]  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$117(ASC), $$118(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC) ]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$139, $$140]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$139(ASC), $$140(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$139, $$140]  |PARTITIONED|
+                -- PRE_CLUSTERED_GROUP_BY[$$126, $$127]  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$126(ASC), $$127(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-3_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-3_ps.plan
index d99b3ed..8fc7dc7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-3_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-3_ps.plan
@@ -1,83 +1,82 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$130, $$131]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$139, $$140]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$139(ASC), $$140(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$139, $$140]  |PARTITIONED|
+                            -- PRE_CLUSTERED_GROUP_BY[$$126, $$127]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$126(ASC), $$127(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$130(ASC), $$131(ASC)]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$130, $$131]  |PARTITIONED|
-                              -- PRE_CLUSTERED_GROUP_BY[$$117, $$118]  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$139, $$140]  |PARTITIONED|
                                       {
                                         -- AGGREGATE  |LOCAL|
-                                          -- STREAM_SELECT  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
                                       }
                                       {
                                         -- AGGREGATE  |LOCAL|
-                                          -- STREAM_SELECT  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
                                       }
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$117(ASC), $$118(ASC)]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$130, $$131]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$130(ASC), $$131(ASC)]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$130, $$131]  |PARTITIONED|
-                                        -- PRE_CLUSTERED_GROUP_BY[$$117, $$118]  |PARTITIONED|
-                                                {
-                                                  -- AGGREGATE  |LOCAL|
-                                                    -- STREAM_SELECT  |LOCAL|
-                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                }
-                                                {
-                                                  -- AGGREGATE  |LOCAL|
-                                                    -- STREAM_SELECT  |LOCAL|
-                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                }
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$117(ASC), $$118(ASC)]  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- STABLE_SORT [$$139(ASC), $$140(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$139, $$140]  |PARTITIONED|
+                                      -- PRE_CLUSTERED_GROUP_BY[$$126, $$127]  |PARTITIONED|
+                                              {
+                                                -- AGGREGATE  |LOCAL|
+                                                  -- STREAM_SELECT  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                              }
+                                              {
+                                                -- AGGREGATE  |LOCAL|
+                                                  -- STREAM_SELECT  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                              }
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STABLE_SORT [$$126(ASC), $$127(ASC)]  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810.plan
index a9398d5..7767cc3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810.plan
@@ -1,37 +1,36 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$140, $$141]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$140(ASC), $$141(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$140, $$141]  |PARTITIONED|
-                  -- PRE_CLUSTERED_GROUP_BY[$$128, $$129]  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$128(ASC), $$129(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC) ]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$150, $$151]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$150(ASC), $$151(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$150, $$151]  |PARTITIONED|
+                -- PRE_CLUSTERED_GROUP_BY[$$138, $$139]  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$138(ASC), $$139(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810_ps.plan
index 038190e..7fb7d4a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810_ps.plan
@@ -1,83 +1,82 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$140, $$141]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$150, $$151]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$150(ASC), $$151(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$150, $$151]  |PARTITIONED|
+                            -- PRE_CLUSTERED_GROUP_BY[$$138, $$139]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$138(ASC), $$139(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$140(ASC), $$141(ASC)]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$140, $$141]  |PARTITIONED|
-                              -- PRE_CLUSTERED_GROUP_BY[$$128, $$129]  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$150, $$151]  |PARTITIONED|
                                       {
                                         -- AGGREGATE  |LOCAL|
-                                          -- STREAM_SELECT  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
                                       }
                                       {
                                         -- AGGREGATE  |LOCAL|
-                                          -- STREAM_SELECT  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
                                       }
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$128(ASC), $$129(ASC)]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$140, $$141]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$140(ASC), $$141(ASC)]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$140, $$141]  |PARTITIONED|
-                                        -- PRE_CLUSTERED_GROUP_BY[$$128, $$129]  |PARTITIONED|
-                                                {
-                                                  -- AGGREGATE  |LOCAL|
-                                                    -- STREAM_SELECT  |LOCAL|
-                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                }
-                                                {
-                                                  -- AGGREGATE  |LOCAL|
-                                                    -- STREAM_SELECT  |LOCAL|
-                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                }
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$128(ASC), $$129(ASC)]  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- STABLE_SORT [$$150(ASC), $$151(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$150, $$151]  |PARTITIONED|
+                                      -- PRE_CLUSTERED_GROUP_BY[$$138, $$139]  |PARTITIONED|
+                                              {
+                                                -- AGGREGATE  |LOCAL|
+                                                  -- STREAM_SELECT  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                              }
+                                              {
+                                                -- AGGREGATE  |LOCAL|
+                                                  -- STREAM_SELECT  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                              }
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STABLE_SORT [$$138(ASC), $$139(ASC)]  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (tpch.LineItem)  |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 ea6589c..079729e 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
@@ -1,137 +1,130 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$cntrycode(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$cntrycode(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- SORT_GROUP_BY[$$176]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- HASH_PARTITION_EXCHANGE [$$176]  |PARTITIONED|
-                          -- SORT_GROUP_BY[$$phone_substr]  |PARTITIONED|
-                                  {
-                                    -- AGGREGATE  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                  }
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- STREAM_SELECT  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- SORT_GROUP_BY[$$173]  |PARTITIONED|
-                                                  {
-                                                    -- AGGREGATE  |LOCAL|
-                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                  }
-                                            -- 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 [$$169(ASC)]  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |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][$$158]  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                          -- UNNEST  |UNPARTITIONED|
-                                                                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                                            -- HASH_PARTITION_EXCHANGE [$$163]  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$cntrycode(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$cntrycode(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- SORT_GROUP_BY[$$176]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- HASH_PARTITION_EXCHANGE [$$176]  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$phone_substr]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- SORT_GROUP_BY[$$176]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- HASH_PARTITION_EXCHANGE [$$176]  |PARTITIONED|
-                                    -- SORT_GROUP_BY[$$phone_substr]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$173]  |PARTITIONED|
                                             {
                                               -- AGGREGATE  |LOCAL|
                                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                                             }
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- STREAM_SELECT  |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 [$$169(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- SORT_GROUP_BY[$$173]  |PARTITIONED|
-                                                            {
-                                                              -- AGGREGATE  |LOCAL|
-                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                            }
-                                                      -- 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 [$$169(ASC)]  |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|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- HYBRID_HASH_JOIN [$$phone_substr][$$158]  |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][$$158]  |PARTITIONED|
-                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                          -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                    -- UNNEST  |UNPARTITIONED|
-                                                                                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                                                      -- HASH_PARTITION_EXCHANGE [$$163]  |PARTITIONED|
-                                                                        -- ASSIGN  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                    -- UNNEST  |UNPARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$163]  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- SORT_GROUP_BY[$$176]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- HASH_PARTITION_EXCHANGE [$$176]  |PARTITIONED|
+                                  -- SORT_GROUP_BY[$$phone_substr]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- SORT_GROUP_BY[$$173]  |PARTITIONED|
+                                                      {
+                                                        -- AGGREGATE  |LOCAL|
+                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                      }
+                                                -- 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 [$$169(ASC)]  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |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][$$158]  |PARTITIONED|
                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                  -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                              -- UNNEST  |UNPARTITIONED|
+                                                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                -- HASH_PARTITION_EXCHANGE [$$163]  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |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/query-issue601.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue601.plan
index 2996ded..2c79baa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue601.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue601.plan
@@ -1,22 +1,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- SORT_GROUP_BY[$$49]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$49]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$46]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- SORT_GROUP_BY[$$52]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$49]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue3316.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue3316.plan
index a7234b3b6..5ffc3f8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue3316.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue3316.plan
@@ -4,484 +4,80 @@
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- STABLE_SORT [$$192(ASC)]  |PARTITIONED|
           -- HASH_PARTITION_EXCHANGE [$$192]  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- PRE_CLUSTERED_GROUP_BY[$$227]  |PARTITIONED|
-                            {
-                              -- 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 [$$227][$$388]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$193]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- STREAM_SELECT  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$193(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- HYBRID_HASH_JOIN [$$193][$$240]  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$193]  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- PRE_CLUSTERED_GROUP_BY[$$227]  |PARTITIONED|
+                          {
+                            -- 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 [$$227][$$388]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$193]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$193(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$193][$$240]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$193]  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$240]  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$240]  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
                                                 -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$126][$$199]  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- HYBRID_HASH_JOIN [$$126][$$199]  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- STREAM_SELECT  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- PRE_CLUSTERED_GROUP_BY[$$225, $$194]  |PARTITIONED|
-                                                                            {
-                                                                              -- AGGREGATE  |LOCAL|
-                                                                                -- STREAM_SELECT  |LOCAL|
-                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                            }
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- STABLE_SORT [$$225(ASC), $$194(ASC)]  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- HYBRID_HASH_JOIN [$$225, $$194][$$254, $$255]  |PARTITIONED|
-                                                                                  -- HASH_PARTITION_EXCHANGE [$$225, $$194]  |PARTITIONED|
-                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- HYBRID_HASH_JOIN [$$223][$$207]  |PARTITIONED|
-                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- PRE_CLUSTERED_GROUP_BY[$$225, $$194]  |PARTITIONED|
+                                                                      {
+                                                                        -- AGGREGATE  |LOCAL|
+                                                                          -- STREAM_SELECT  |LOCAL|
+                                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                      }
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- STABLE_SORT [$$225(ASC), $$194(ASC)]  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- HYBRID_HASH_JOIN [$$225, $$194][$$254, $$255]  |PARTITIONED|
+                                                                            -- HASH_PARTITION_EXCHANGE [$$225, $$194]  |PARTITIONED|
+                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- HYBRID_HASH_JOIN [$$223][$$207]  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                            -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
                                                                                               -- ASSIGN  |PARTITIONED|
                                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                   -- REPLICATE  |PARTITIONED|
-                                                                                                    -- HASH_PARTITION_EXCHANGE [$$417]  |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|
-                                                                                                                      -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
-                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                          -- HASH_PARTITION_EXCHANGE [$$207]  |PARTITIONED|
-                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                              -- STREAM_SELECT  |PARTITIONED|
-                                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                        -- REPLICATE  |PARTITIONED|
-                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                            -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                        -- STREAM_SELECT  |PARTITIONED|
-                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                            -- UNNEST  |PARTITIONED|
-                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                -- PRE_CLUSTERED_GROUP_BY[$$238, $$235]  |PARTITIONED|
-                                                                                                        {
-                                                                                                          -- AGGREGATE  |LOCAL|
-                                                                                                            -- AGGREGATE  |LOCAL|
-                                                                                                              -- STREAM_SELECT  |LOCAL|
-                                                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                        }
-                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                    -- STABLE_SORT [$$238(ASC), $$235(ASC)]  |PARTITIONED|
-                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                            -- HYBRID_HASH_JOIN [$$238, $$235][$$252, $$249]  |PARTITIONED|
-                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                      -- REPLICATE  |PARTITIONED|
-                                                                                                                        -- HASH_PARTITION_EXCHANGE [$$425, $$426]  |PARTITIONED|
-                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                              -- HYBRID_HASH_JOIN [$$428][$$427]  |PARTITIONED|
-                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                        -- REPLICATE  |PARTITIONED|
-                                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$417]  |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|
-                                                                                                                                                            -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
-                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                -- HASH_PARTITION_EXCHANGE [$$427]  |PARTITIONED|
-                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                        -- REPLICATE  |PARTITIONED|
-                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                            -- ASSIGN  |PARTITIONED|
-                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                    -- REPLICATE  |PARTITIONED|
-                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                        -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                              -- HASH_PARTITION_EXCHANGE [$$252, $$249]  |PARTITIONED|
-                                                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                      -- HYBRID_HASH_JOIN [$$135][$$198]  |PARTITIONED|
-                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                              -- HYBRID_HASH_JOIN [$$243][$$210]  |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|
-                                                                                                                                                -- HYBRID_HASH_JOIN [$$445][$$448]  |PARTITIONED|
-                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                          -- REPLICATE  |PARTITIONED|
-                                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$417]  |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|
-                                                                                                                                                                              -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
-                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$448]  |PARTITIONED|
-                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                          -- REPLICATE  |PARTITIONED|
-                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                          -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                -- HASH_PARTITION_EXCHANGE [$$210]  |PARTITIONED|
-                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                      -- ASSIGN  |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|
-                                                                                                                                                            -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                          -- UNNEST  |UNPARTITIONED|
-                                                                                                                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                            -- UNNEST  |UNPARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$388]  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- HYBRID_HASH_JOIN [$$162][$$202]  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- HYBRID_HASH_JOIN [$$165][$$201]  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- PRE_CLUSTERED_GROUP_BY[$$256, $$196]  |PARTITIONED|
-                                                                    {
-                                                                      -- AGGREGATE  |LOCAL|
-                                                                        -- STREAM_SELECT  |LOCAL|
-                                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                    }
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- STABLE_SORT [$$256(ASC), $$196(ASC)]  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- HYBRID_HASH_JOIN [$$256, $$196][$$459, $$460]  |PARTITIONED|
-                                                                          -- HASH_PARTITION_EXCHANGE [$$256, $$196]  |PARTITIONED|
-                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- HYBRID_HASH_JOIN [$$260][$$214]  |PARTITIONED|
-                                                                                  -- HASH_PARTITION_EXCHANGE [$$260]  |PARTITIONED|
-                                                                                    -- PRE_CLUSTERED_GROUP_BY[$$257]  |PARTITIONED|
-                                                                                            {
-                                                                                              -- AGGREGATE  |LOCAL|
-                                                                                                -- STREAM_SELECT  |LOCAL|
-                                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                            }
-                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- STABLE_SORT [$$257(ASC)]  |PARTITIONED|
-                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                -- HYBRID_HASH_JOIN [$$257][$$261]  |PARTITIONED|
-                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                       -- ASSIGN  |PARTITIONED|
                                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          -- REPLICATE  |PARTITIONED|
-                                                                                                            -- HASH_PARTITION_EXCHANGE [$$399]  |PARTITIONED|
-                                                                                                              -- REPLICATE  |PARTITIONED|
-                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                        -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
-                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                  -- HASH_PARTITION_EXCHANGE [$$261]  |PARTITIONED|
-                                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          -- HYBRID_HASH_JOIN [$$266][$$265]  |PARTITIONED|
+                                                                                                          -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
                                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$268, $$270]  |PARTITIONED|
-                                                                                                                              {
-                                                                                                                                -- AGGREGATE  |LOCAL|
-                                                                                                                                  -- STREAM_SELECT  |LOCAL|
-                                                                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                                              }
-                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                          -- STABLE_SORT [$$268(ASC), $$270(ASC)]  |PARTITIONED|
-                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                  -- HYBRID_HASH_JOIN [$$268, $$270][$$278, $$279]  |PARTITIONED|
-                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                        -- ASSIGN  |PARTITIONED|
-                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                            -- REPLICATE  |PARTITIONED|
-                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$409, $$411]  |PARTITIONED|
-                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                    -- HYBRID_HASH_JOIN [$$417][$$418]  |PARTITIONED|
-                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                        -- REPLICATE  |PARTITIONED|
-                                                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$417]  |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|
-                                                                                                                                                                            -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
-                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$418]  |PARTITIONED|
-                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                          -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                            -- ASSIGN  |PARTITIONED|
-                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                    -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                          -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                              -- UNNEST  |PARTITIONED|
-                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                  -- PRE_CLUSTERED_GROUP_BY[$$284, $$285]  |PARTITIONED|
-                                                                                                                                                          {
-                                                                                                                                                            -- AGGREGATE  |LOCAL|
-                                                                                                                                                              -- AGGREGATE  |LOCAL|
-                                                                                                                                                                -- STREAM_SELECT  |LOCAL|
-                                                                                                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                                                                          }
-                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                      -- STABLE_SORT [$$284(ASC), $$285(ASC)]  |PARTITIONED|
-                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                              -- HYBRID_HASH_JOIN [$$284, $$285][$$294, $$295]  |PARTITIONED|
-                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$425, $$426]  |PARTITIONED|
-                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                -- HYBRID_HASH_JOIN [$$428][$$427]  |PARTITIONED|
-                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                          -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$417]  |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|
-                                                                                                                                                                                                              -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
-                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$427]  |PARTITIONED|
-                                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                          -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                          -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                -- HASH_PARTITION_EXCHANGE [$$294, $$295]  |PARTITIONED|
-                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                        -- HYBRID_HASH_JOIN [$$302][$$301]  |PARTITIONED|
-                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                -- HYBRID_HASH_JOIN [$$304][$$303]  |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|
-                                                                                                                                                                                                  -- HYBRID_HASH_JOIN [$$445][$$448]  |PARTITIONED|
-                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$417]  |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|
-                                                                                                                                                                                                                                -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
-                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$448]  |PARTITIONED|
-                                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                        -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                            -- DATASOURCE_SCAN (test.collection1)  |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 [$$444]  |PARTITIONED|
-                                                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                -- STREAM_SELECT  |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|
-                                                                                                                                                                                                                    -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                            -- UNNEST  |UNPARTITIONED|
-                                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                                                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                              -- UNNEST  |UNPARTITIONED|
-                                                                                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                                                                  -- HASH_PARTITION_EXCHANGE [$$214]  |PARTITIONED|
-                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                    -- HASH_PARTITION_EXCHANGE [$$207]  |PARTITIONED|
+                                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                                        -- ASSIGN  |PARTITIONED|
                                                                                           -- ASSIGN  |PARTITIONED|
                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                               -- REPLICATE  |PARTITIONED|
@@ -489,82 +85,643 @@
                                                                                                   -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                       -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
                                                                                 -- STREAM_SELECT  |PARTITIONED|
-                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                    -- UNNEST  |PARTITIONED|
-                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- PRE_CLUSTERED_GROUP_BY[$$327, $$326]  |PARTITIONED|
-                                                                                                {
+                                                                                  -- UNNEST  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$238, $$235]  |PARTITIONED|
+                                                                                              {
+                                                                                                -- AGGREGATE  |LOCAL|
                                                                                                   -- AGGREGATE  |LOCAL|
-                                                                                                    -- AGGREGATE  |LOCAL|
-                                                                                                      -- STREAM_SELECT  |LOCAL|
-                                                                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                }
-                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            -- STABLE_SORT [$$327(ASC), $$326(ASC)]  |PARTITIONED|
-                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                    -- HYBRID_HASH_JOIN [$$327, $$326][$$398, $$397]  |PARTITIONED|
-                                                                                                      -- HASH_PARTITION_EXCHANGE [$$327, $$326]  |PARTITIONED|
+                                                                                                    -- STREAM_SELECT  |LOCAL|
+                                                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                              }
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- STABLE_SORT [$$238(ASC), $$235(ASC)]  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- HYBRID_HASH_JOIN [$$238, $$235][$$252, $$249]  |PARTITIONED|
+                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                            -- HASH_PARTITION_EXCHANGE [$$425, $$426]  |PARTITIONED|
+                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- HYBRID_HASH_JOIN [$$428][$$427]  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
+                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$427]  |PARTITIONED|
+                                                                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                        -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                    -- HASH_PARTITION_EXCHANGE [$$252, $$249]  |PARTITIONED|
+                                                                                                      -- ASSIGN  |PARTITIONED|
                                                                                                         -- STREAM_PROJECT  |PARTITIONED|
                                                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                            -- HYBRID_HASH_JOIN [$$320][$$319]  |PARTITIONED|
-                                                                                                              -- HASH_PARTITION_EXCHANGE [$$320]  |PARTITIONED|
-                                                                                                                -- PRE_CLUSTERED_GROUP_BY[$$328]  |PARTITIONED|
-                                                                                                                        {
-                                                                                                                          -- AGGREGATE  |LOCAL|
-                                                                                                                            -- STREAM_SELECT  |LOCAL|
-                                                                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                                        }
+                                                                                                            -- HYBRID_HASH_JOIN [$$135][$$198]  |PARTITIONED|
+                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
                                                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                    -- STABLE_SORT [$$328(ASC)]  |PARTITIONED|
+                                                                                                                    -- HYBRID_HASH_JOIN [$$243][$$210]  |PARTITIONED|
                                                                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                        -- ASSIGN  |PARTITIONED|
                                                                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                            -- HYBRID_HASH_JOIN [$$328][$$331]  |PARTITIONED|
+                                                                                                                            -- REPLICATE  |PARTITIONED|
                                                                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                 -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                    -- HYBRID_HASH_JOIN [$$445][$$448]  |PARTITIONED|
+                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
+                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                            -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$448]  |PARTITIONED|
+                                                                                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                          -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$210]  |PARTITIONED|
+                                                                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                            -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                -- UNNEST  |UNPARTITIONED|
+                                                                                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- UNNEST  |UNPARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$388]  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$162][$$202]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$165][$$201]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_SELECT  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- PRE_CLUSTERED_GROUP_BY[$$256, $$196]  |PARTITIONED|
+                                                              {
+                                                                -- AGGREGATE  |LOCAL|
+                                                                  -- STREAM_SELECT  |LOCAL|
+                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                              }
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- STABLE_SORT [$$256(ASC), $$196(ASC)]  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- HYBRID_HASH_JOIN [$$256, $$196][$$459, $$460]  |PARTITIONED|
+                                                                    -- HASH_PARTITION_EXCHANGE [$$256, $$196]  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- HYBRID_HASH_JOIN [$$260][$$214]  |PARTITIONED|
+                                                                            -- HASH_PARTITION_EXCHANGE [$$260]  |PARTITIONED|
+                                                                              -- PRE_CLUSTERED_GROUP_BY[$$257]  |PARTITIONED|
+                                                                                      {
+                                                                                        -- AGGREGATE  |LOCAL|
+                                                                                          -- STREAM_SELECT  |LOCAL|
+                                                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                      }
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- STABLE_SORT [$$257(ASC)]  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- HYBRID_HASH_JOIN [$$257][$$261]  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                    -- HASH_PARTITION_EXCHANGE [$$399]  |PARTITIONED|
+                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                            -- HASH_PARTITION_EXCHANGE [$$261]  |PARTITIONED|
+                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- HYBRID_HASH_JOIN [$$266][$$265]  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- PRE_CLUSTERED_GROUP_BY[$$268, $$270]  |PARTITIONED|
+                                                                                                                      {
+                                                                                                                        -- AGGREGATE  |LOCAL|
+                                                                                                                          -- STREAM_SELECT  |LOCAL|
+                                                                                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                      }
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- STABLE_SORT [$$268(ASC), $$270(ASC)]  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- HYBRID_HASH_JOIN [$$268, $$270][$$278, $$279]  |PARTITIONED|
+                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$409, $$411]  |PARTITIONED|
+                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- HYBRID_HASH_JOIN [$$417][$$418]  |PARTITIONED|
+                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                                                -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
+                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                              -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$418]  |PARTITIONED|
+                                                                                                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                        -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                  -- UNNEST  |PARTITIONED|
                                                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                      -- REPLICATE  |PARTITIONED|
-                                                                                                                                        -- HASH_PARTITION_EXCHANGE [$$399]  |PARTITIONED|
-                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$284, $$285]  |PARTITIONED|
+                                                                                                                                              {
+                                                                                                                                                -- AGGREGATE  |LOCAL|
+                                                                                                                                                  -- AGGREGATE  |LOCAL|
+                                                                                                                                                    -- STREAM_SELECT  |LOCAL|
+                                                                                                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                              }
+                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- STABLE_SORT [$$284(ASC), $$285(ASC)]  |PARTITIONED|
                                                                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                               -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                  -- HYBRID_HASH_JOIN [$$284, $$285][$$294, $$295]  |PARTITIONED|
+                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$425, $$426]  |PARTITIONED|
+                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                  -- HYBRID_HASH_JOIN [$$428][$$427]  |PARTITIONED|
+                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
+                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                          -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$427]  |PARTITIONED|
+                                                                                                                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                        -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$294, $$295]  |PARTITIONED|
+                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                            -- HYBRID_HASH_JOIN [$$302][$$301]  |PARTITIONED|
+                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                    -- HYBRID_HASH_JOIN [$$304][$$303]  |PARTITIONED|
+                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                    -- HYBRID_HASH_JOIN [$$445][$$448]  |PARTITIONED|
+                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
+                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                            -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$448]  |PARTITIONED|
+                                                                                                                                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                          -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$444]  |PARTITIONED|
+                                                                                                                                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                  -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                -- UNNEST  |UNPARTITIONED|
+                                                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                        -- UNNEST  |UNPARTITIONED|
+                                                                                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                            -- HASH_PARTITION_EXCHANGE [$$214]  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                          -- UNNEST  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- PRE_CLUSTERED_GROUP_BY[$$327, $$326]  |PARTITIONED|
+                                                                                      {
+                                                                                        -- AGGREGATE  |LOCAL|
+                                                                                          -- AGGREGATE  |LOCAL|
+                                                                                            -- STREAM_SELECT  |LOCAL|
+                                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                      }
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- STABLE_SORT [$$327(ASC), $$326(ASC)]  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- HYBRID_HASH_JOIN [$$327, $$326][$$398, $$397]  |PARTITIONED|
+                                                                                            -- HASH_PARTITION_EXCHANGE [$$327, $$326]  |PARTITIONED|
+                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- HYBRID_HASH_JOIN [$$320][$$319]  |PARTITIONED|
+                                                                                                    -- HASH_PARTITION_EXCHANGE [$$320]  |PARTITIONED|
+                                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$328]  |PARTITIONED|
+                                                                                                              {
+                                                                                                                -- AGGREGATE  |LOCAL|
+                                                                                                                  -- STREAM_SELECT  |LOCAL|
+                                                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                              }
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- STABLE_SORT [$$328(ASC)]  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- HYBRID_HASH_JOIN [$$328][$$331]  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$399]  |PARTITIONED|
+                                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                      -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$331]  |PARTITIONED|
+                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                            -- HYBRID_HASH_JOIN [$$336][$$335]  |PARTITIONED|
+                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$338, $$340]  |PARTITIONED|
+                                                                                                                                              {
+                                                                                                                                                -- AGGREGATE  |LOCAL|
+                                                                                                                                                  -- STREAM_SELECT  |LOCAL|
+                                                                                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                              }
+                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- STABLE_SORT [$$338(ASC), $$340(ASC)]  |PARTITIONED|
+                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                  -- HYBRID_HASH_JOIN [$$338, $$340][$$348, $$349]  |PARTITIONED|
+                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$409, $$411]  |PARTITIONED|
+                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                  -- HYBRID_HASH_JOIN [$$417][$$418]  |PARTITIONED|
+                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                        -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
+                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                      -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$418]  |PARTITIONED|
+                                                                                                                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                          -- UNNEST  |PARTITIONED|
+                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                              -- PRE_CLUSTERED_GROUP_BY[$$354, $$355]  |PARTITIONED|
+                                                                                                                                                                      {
+                                                                                                                                                                        -- AGGREGATE  |LOCAL|
+                                                                                                                                                                          -- AGGREGATE  |LOCAL|
+                                                                                                                                                                            -- STREAM_SELECT  |LOCAL|
+                                                                                                                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                                                      }
+                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                  -- STABLE_SORT [$$354(ASC), $$355(ASC)]  |PARTITIONED|
+                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                          -- HYBRID_HASH_JOIN [$$354, $$355][$$364, $$365]  |PARTITIONED|
+                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$425, $$426]  |PARTITIONED|
+                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                          -- HYBRID_HASH_JOIN [$$428][$$427]  |PARTITIONED|
+                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
+                                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                  -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$427]  |PARTITIONED|
+                                                                                                                                                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$364, $$365]  |PARTITIONED|
+                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                    -- HYBRID_HASH_JOIN [$$372][$$371]  |PARTITIONED|
+                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                            -- HYBRID_HASH_JOIN [$$374][$$373]  |PARTITIONED|
+                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                            -- HYBRID_HASH_JOIN [$$445][$$448]  |PARTITIONED|
+                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
+                                                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                    -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$448]  |PARTITIONED|
+                                                                                                                                                                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                  -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$444]  |PARTITIONED|
+                                                                                                                                                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                          -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                        -- UNNEST  |UNPARTITIONED|
+                                                                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                -- UNNEST  |UNPARTITIONED|
+                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                                    -- HASH_PARTITION_EXCHANGE [$$319]  |PARTITIONED|
+                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                        -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                            -- HASH_PARTITION_EXCHANGE [$$398, $$397]  |PARTITIONED|
+                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- HYBRID_HASH_JOIN [$$172][$$200]  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                            -- HYBRID_HASH_JOIN [$$391][$$216]  |PARTITIONED|
+                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                    -- HYBRID_HASH_JOIN [$$391][$$390]  |PARTITIONED|
+                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$391]  |PARTITIONED|
+                                                                                                                        -- PRE_CLUSTERED_GROUP_BY[$$399]  |PARTITIONED|
+                                                                                                                                {
+                                                                                                                                  -- AGGREGATE  |LOCAL|
+                                                                                                                                    -- STREAM_SELECT  |LOCAL|
+                                                                                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                }
+                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                            -- STABLE_SORT [$$399(ASC)]  |PARTITIONED|
+                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                    -- HYBRID_HASH_JOIN [$$399][$$402]  |PARTITIONED|
+                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$399]  |PARTITIONED|
+                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                                 -- ASSIGN  |PARTITIONED|
                                                                                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                                     -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
                                                                                                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$331]  |PARTITIONED|
-                                                                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                      -- HYBRID_HASH_JOIN [$$336][$$335]  |PARTITIONED|
-                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$402]  |PARTITIONED|
+                                                                                                                                        -- ASSIGN  |PARTITIONED|
                                                                                                                                           -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                            -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                              -- HYBRID_HASH_JOIN [$$407][$$406]  |PARTITIONED|
                                                                                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                  -- PRE_CLUSTERED_GROUP_BY[$$338, $$340]  |PARTITIONED|
-                                                                                                                                                          {
-                                                                                                                                                            -- AGGREGATE  |LOCAL|
-                                                                                                                                                              -- STREAM_SELECT  |LOCAL|
-                                                                                                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                                                                          }
-                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                      -- STABLE_SORT [$$338(ASC), $$340(ASC)]  |PARTITIONED|
-                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                              -- HYBRID_HASH_JOIN [$$338, $$340][$$348, $$349]  |PARTITIONED|
-                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                        -- PRE_CLUSTERED_GROUP_BY[$$409, $$411]  |PARTITIONED|
+                                                                                                                                                                {
+                                                                                                                                                                  -- AGGREGATE  |LOCAL|
+                                                                                                                                                                    -- STREAM_SELECT  |LOCAL|
+                                                                                                                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                                                }
+                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                            -- STABLE_SORT [$$409(ASC), $$411(ASC)]  |PARTITIONED|
+                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                    -- HYBRID_HASH_JOIN [$$409, $$411][$$419, $$420]  |PARTITIONED|
                                                                                                                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                                                         -- REPLICATE  |PARTITIONED|
                                                                                                                                                                           -- HASH_PARTITION_EXCHANGE [$$409, $$411]  |PARTITIONED|
@@ -574,151 +731,110 @@
                                                                                                                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                                                                     -- REPLICATE  |PARTITIONED|
                                                                                                                                                                                       -- HASH_PARTITION_EXCHANGE [$$417]  |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|
-                                                                                                                                                                                                        -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
-                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$418]  |PARTITIONED|
-                                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                      -- STREAM_SELECT  |PARTITIONED|
                                                                                                                                                                                         -- ASSIGN  |PARTITIONED|
                                                                                                                                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                                                                             -- REPLICATE  |PARTITIONED|
                                                                                                                                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                          -- UNNEST  |PARTITIONED|
-                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                              -- PRE_CLUSTERED_GROUP_BY[$$354, $$355]  |PARTITIONED|
-                                                                                                                                                                                      {
-                                                                                                                                                                                        -- AGGREGATE  |LOCAL|
-                                                                                                                                                                                          -- AGGREGATE  |LOCAL|
-                                                                                                                                                                                            -- STREAM_SELECT  |LOCAL|
-                                                                                                                                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                                                                                                      }
-                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                  -- STABLE_SORT [$$354(ASC), $$355(ASC)]  |PARTITIONED|
-                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                          -- HYBRID_HASH_JOIN [$$354, $$355][$$364, $$365]  |PARTITIONED|
-                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
                                                                                                                                                                                                 -- ASSIGN  |PARTITIONED|
                                                                                                                                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$425, $$426]  |PARTITIONED|
-                                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                    -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$418]  |PARTITIONED|
+                                                                                                                                                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                              -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                            -- UNNEST  |PARTITIONED|
+                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                -- PRE_CLUSTERED_GROUP_BY[$$425, $$426]  |PARTITIONED|
+                                                                                                                                                                                        {
+                                                                                                                                                                                          -- AGGREGATE  |LOCAL|
+                                                                                                                                                                                            -- AGGREGATE  |LOCAL|
+                                                                                                                                                                                              -- STREAM_SELECT  |LOCAL|
+                                                                                                                                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                                                                        }
+                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                    -- STABLE_SORT [$$425(ASC), $$426(ASC)]  |PARTITIONED|
+                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                            -- HYBRID_HASH_JOIN [$$425, $$426][$$435, $$436]  |PARTITIONED|
+                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$425, $$426]  |PARTITIONED|
+                                                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                        -- HYBRID_HASH_JOIN [$$428][$$427]  |PARTITIONED|
                                                                                                                                                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                            -- HYBRID_HASH_JOIN [$$428][$$427]  |PARTITIONED|
+                                                                                                                                                                                                            -- ASSIGN  |PARTITIONED|
                                                                                                                                                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                        -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
-                                                                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
+                                                                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                                                                                                             -- ASSIGN  |PARTITIONED|
                                                                                                                                                                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
                                                                                                                                                                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$427]  |PARTITIONED|
+                                                                                                                                                                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                              -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$435, $$436]  |PARTITIONED|
+                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                      -- HYBRID_HASH_JOIN [$$443][$$442]  |PARTITIONED|
+                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                              -- HYBRID_HASH_JOIN [$$445][$$444]  |PARTITIONED|
+                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                          -- HYBRID_HASH_JOIN [$$445][$$448]  |PARTITIONED|
+                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
                                                                                                                                                                                                                                       -- ASSIGN  |PARTITIONED|
                                                                                                                                                                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                          -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                                                          -- REPLICATE  |PARTITIONED|
                                                                                                                                                                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$427]  |PARTITIONED|
-                                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                  -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                      -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$364, $$365]  |PARTITIONED|
-                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                    -- HYBRID_HASH_JOIN [$$372][$$371]  |PARTITIONED|
-                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                            -- HYBRID_HASH_JOIN [$$374][$$373]  |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|
-                                                                                                                                                                                                                              -- HYBRID_HASH_JOIN [$$445][$$448]  |PARTITIONED|
-                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
-                                                                                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
                                                                                                                                                                                                                                               -- ASSIGN  |PARTITIONED|
                                                                                                                                                                                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                                  -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
                                                                                                                                                                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                            -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
-                                                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                                                                                -- HASH_PARTITION_EXCHANGE [$$448]  |PARTITIONED|
-                                                                                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                            -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                        -- DATASOURCE_SCAN (test.collection1)  |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 [$$444]  |PARTITIONED|
-                                                                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                            -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$448]  |PARTITIONED|
+                                                                                                                                                                                                                              -- STREAM_SELECT  |PARTITIONED|
                                                                                                                                                                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                                                                                                                   -- REPLICATE  |PARTITIONED|
                                                                                                                                                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
                                                                                                                                                                                                                                         -- ASSIGN  |PARTITIONED|
                                                                                                                                                                                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                                                                                                                             -- REPLICATE  |PARTITIONED|
@@ -726,281 +842,59 @@
                                                                                                                                                                                                                                                 -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
                                                                                                                                                                                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                                                                                                                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                        -- UNNEST  |UNPARTITIONED|
-                                                                                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                                                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                                          -- UNNEST  |UNPARTITIONED|
-                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                                                                                              -- HASH_PARTITION_EXCHANGE [$$319]  |PARTITIONED|
-                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$444]  |PARTITIONED|
+                                                                                                                                                                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                        -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                          -- UNNEST  |UNPARTITIONED|
+                                                                                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                                  -- UNNEST  |UNPARTITIONED|
+                                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$390]  |PARTITIONED|
+                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                              -- HASH_PARTITION_EXCHANGE [$$216]  |PARTITIONED|
+                                                                                                                -- ASSIGN  |PARTITIONED|
                                                                                                                   -- ASSIGN  |PARTITIONED|
                                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                       -- REPLICATE  |PARTITIONED|
                                                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                           -- ASSIGN  |PARTITIONED|
-                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                  -- REPLICATE  |PARTITIONED|
-                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                      -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                      -- HASH_PARTITION_EXCHANGE [$$398, $$397]  |PARTITIONED|
-                                                                                                        -- ASSIGN  |PARTITIONED|
-                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                              -- HYBRID_HASH_JOIN [$$172][$$200]  |PARTITIONED|
-                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                      -- HYBRID_HASH_JOIN [$$391][$$216]  |PARTITIONED|
-                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
                                                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                              -- HYBRID_HASH_JOIN [$$391][$$390]  |PARTITIONED|
-                                                                                                                                -- HASH_PARTITION_EXCHANGE [$$391]  |PARTITIONED|
-                                                                                                                                  -- PRE_CLUSTERED_GROUP_BY[$$399]  |PARTITIONED|
-                                                                                                                                          {
-                                                                                                                                            -- AGGREGATE  |LOCAL|
-                                                                                                                                              -- STREAM_SELECT  |LOCAL|
-                                                                                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                                                          }
+                                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                  -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
                                                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                      -- STABLE_SORT [$$399(ASC)]  |PARTITIONED|
-                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                              -- HYBRID_HASH_JOIN [$$399][$$402]  |PARTITIONED|
-                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                  -- REPLICATE  |PARTITIONED|
-                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$399]  |PARTITIONED|
-                                                                                                                                                      -- REPLICATE  |PARTITIONED|
-                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                            -- ASSIGN  |PARTITIONED|
-                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
-                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                -- HASH_PARTITION_EXCHANGE [$$402]  |PARTITIONED|
-                                                                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                        -- HYBRID_HASH_JOIN [$$407][$$406]  |PARTITIONED|
-                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                              -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                    -- PRE_CLUSTERED_GROUP_BY[$$409, $$411]  |PARTITIONED|
-                                                                                                                                                                            {
-                                                                                                                                                                              -- AGGREGATE  |LOCAL|
-                                                                                                                                                                                -- STREAM_SELECT  |LOCAL|
-                                                                                                                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                                                                                            }
-                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                        -- STABLE_SORT [$$409(ASC), $$411(ASC)]  |PARTITIONED|
-                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                -- HYBRID_HASH_JOIN [$$409, $$411][$$419, $$420]  |PARTITIONED|
-                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$409, $$411]  |PARTITIONED|
-                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                            -- HYBRID_HASH_JOIN [$$417][$$418]  |PARTITIONED|
-                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$417]  |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|
-                                                                                                                                                                                                                    -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
-                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$418]  |PARTITIONED|
-                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                  -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                            -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                        -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                            -- UNNEST  |PARTITIONED|
-                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                -- PRE_CLUSTERED_GROUP_BY[$$425, $$426]  |PARTITIONED|
-                                                                                                                                                                                                        {
-                                                                                                                                                                                                          -- AGGREGATE  |LOCAL|
-                                                                                                                                                                                                            -- AGGREGATE  |LOCAL|
-                                                                                                                                                                                                              -- STREAM_SELECT  |LOCAL|
-                                                                                                                                                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                                                                                                                        }
-                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                    -- STABLE_SORT [$$425(ASC), $$426(ASC)]  |PARTITIONED|
-                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                            -- HYBRID_HASH_JOIN [$$425, $$426][$$435, $$436]  |PARTITIONED|
-                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$425, $$426]  |PARTITIONED|
-                                                                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                        -- HYBRID_HASH_JOIN [$$428][$$427]  |PARTITIONED|
-                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$417]  |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|
-                                                                                                                                                                                                                                                      -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
-                                                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$427]  |PARTITIONED|
-                                                                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                              -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                              -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                  -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$435, $$436]  |PARTITIONED|
-                                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                      -- HYBRID_HASH_JOIN [$$443][$$442]  |PARTITIONED|
-                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                              -- HYBRID_HASH_JOIN [$$445][$$444]  |PARTITIONED|
-                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                          -- HYBRID_HASH_JOIN [$$445][$$448]  |PARTITIONED|
-                                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$417]  |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|
-                                                                                                                                                                                                                                                                        -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
-                                                                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$448]  |PARTITIONED|
-                                                                                                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                                                -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                                                            -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                                -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                                    -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$444]  |PARTITIONED|
-                                                                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                                        -- STREAM_SELECT  |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|
-                                                                                                                                                                                                                                                            -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                          -- UNNEST  |UNPARTITIONED|
-                                                                                                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                                                                                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                                                            -- UNNEST  |UNPARTITIONED|
-                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                                                                                                                -- HASH_PARTITION_EXCHANGE [$$390]  |PARTITIONED|
-                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                        -- REPLICATE  |PARTITIONED|
-                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                            -- ASSIGN  |PARTITIONED|
-                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                    -- REPLICATE  |PARTITIONED|
-                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                        -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                        -- HASH_PARTITION_EXCHANGE [$$216]  |PARTITIONED|
-                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                            -- ASSIGN  |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|
-                                                                                                                                                  -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                  -- UNNEST  |UNPARTITIONED|
-                                                                                                                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                    -- UNNEST  |UNPARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                            -- UNNEST  |UNPARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                        -- UNNEST  |UNPARTITIONED|
+                                                                                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        -- UNNEST  |UNPARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
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 11bad6c..a73a314 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
@@ -16,17 +16,15 @@
               -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$47][$$49]  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$47]  |PARTITIONED|
-                -- DATASOURCE_SCAN (test.s)  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$49]  |PARTITIONED|
-                -- STREAM_PROJECT  |UNPARTITIONED|
-                  -- ASSIGN  |UNPARTITIONED|
-                    -- UNNEST  |UNPARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
\ No newline at end of file
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- HYBRID_HASH_JOIN [$$47][$$49]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$47]  |PARTITIONED|
+              -- DATASOURCE_SCAN (test.s)  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$49]  |PARTITIONED|
+              -- ASSIGN  |UNPARTITIONED|
+                -- UNNEST  |UNPARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
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 90e2808..131d733 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
@@ -16,18 +16,17 @@
               -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$48][$$47]  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
-                -- ASSIGN  |UNPARTITIONED|
-                  -- UNNEST  |UNPARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$47]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- DATASOURCE_SCAN (test.s)  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- HYBRID_HASH_JOIN [$$48][$$47]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
+              -- ASSIGN  |UNPARTITIONED|
+                -- UNNEST  |UNPARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$47]  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |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/remove_listify.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/remove_listify.plan
index 96d4248..0a88e9b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/remove_listify.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/remove_listify.plan
@@ -1,36 +1,33 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- PRE_CLUSTERED_GROUP_BY[$$141]  |PARTITIONED|
-                    {
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$141]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
                       -- AGGREGATE  |LOCAL|
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- ASSIGN  |LOCAL|
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                    }
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$141(ASC)]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$141]  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- HYBRID_HASH_JOIN [$$146][$$148]  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$146]  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
+                        -- STREAM_SELECT  |LOCAL|
+                          -- ASSIGN  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$141(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$141]  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$146][$$148]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$146]  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$148]  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$148]  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
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 e815859..09b9dbe 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
@@ -1,28 +1,24 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$#1(ASC), $#2(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$#1(ASC), $#2(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$#1(ASC), $#2(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$#1(ASC), $#2(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
               -- STREAM_PROJECT  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |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|
-                                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$92]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$91][$$92]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$91]  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$92]  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |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 d7c9f38..fdeccc1 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
@@ -1,44 +1,37 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$#1(ASC), $#2(ASC), $#3(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$#1(ASC), $#2(ASC), $#3(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$#1(ASC), $#2(ASC), $#3(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$#1(ASC), $#2(ASC), $#3(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
               -- STREAM_PROJECT  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |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|
-                                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$151][$$152]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$151]  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$152][$$153]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$152]  |PARTITIONED|
+                                  -- ASSIGN  |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|
-                                                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$153]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$153]  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-01.plan
index 30be777..26f64fe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-01.plan
@@ -1,15 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- BTREE_SEARCH (IndexGeoJSON.Geometries.Geometries)  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- RTREE_SEARCH (IndexGeoJSON.Geometries.geomIndex)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- BTREE_SEARCH (IndexGeoJSON.Geometries.Geometries)  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- RTREE_SEARCH (IndexGeoJSON.Geometries.geomIndex)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-02.plan
index 30be777..26f64fe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-02.plan
@@ -1,15 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- BTREE_SEARCH (IndexGeoJSON.Geometries.Geometries)  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- RTREE_SEARCH (IndexGeoJSON.Geometries.geomIndex)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- BTREE_SEARCH (IndexGeoJSON.Geometries.Geometries)  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- RTREE_SEARCH (IndexGeoJSON.Geometries.geomIndex)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-03.plan
index 30be777..26f64fe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-03.plan
@@ -1,15 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- BTREE_SEARCH (IndexGeoJSON.Geometries.Geometries)  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- RTREE_SEARCH (IndexGeoJSON.Geometries.geomIndex)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- BTREE_SEARCH (IndexGeoJSON.Geometries.Geometries)  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- RTREE_SEARCH (IndexGeoJSON.Geometries.geomIndex)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-04.plan
index 30be777..26f64fe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-04.plan
@@ -1,15 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- BTREE_SEARCH (IndexGeoJSON.Geometries.Geometries)  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- RTREE_SEARCH (IndexGeoJSON.Geometries.geomIndex)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- BTREE_SEARCH (IndexGeoJSON.Geometries.Geometries)  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- RTREE_SEARCH (IndexGeoJSON.Geometries.geomIndex)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-05.plan
index 30be777..26f64fe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-05.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-05.plan
@@ -1,15 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- BTREE_SEARCH (IndexGeoJSON.Geometries.Geometries)  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- RTREE_SEARCH (IndexGeoJSON.Geometries.geomIndex)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- BTREE_SEARCH (IndexGeoJSON.Geometries.Geometries)  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- RTREE_SEARCH (IndexGeoJSON.Geometries.geomIndex)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-06.plan
index 30be777..26f64fe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-06.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-06.plan
@@ -1,15 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- BTREE_SEARCH (IndexGeoJSON.Geometries.Geometries)  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- RTREE_SEARCH (IndexGeoJSON.Geometries.geomIndex)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- BTREE_SEARCH (IndexGeoJSON.Geometries.Geometries)  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- RTREE_SEARCH (IndexGeoJSON.Geometries.geomIndex)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-07.plan
index 30be777..26f64fe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-07.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/rtree-sidx-idxonly-07.plan
@@ -1,15 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- BTREE_SEARCH (IndexGeoJSON.Geometries.Geometries)  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- RTREE_SEARCH (IndexGeoJSON.Geometries.geomIndex)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- BTREE_SEARCH (IndexGeoJSON.Geometries.Geometries)  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- RTREE_SEARCH (IndexGeoJSON.Geometries.geomIndex)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-01.plan
index 7cd72c2..e981647 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-01.plan
@@ -1,8 +1,7 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-02.plan
index 7cd72c2..e981647 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-02.plan
@@ -1,8 +1,7 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-03.plan
index 7cd72c2..e981647 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-03.plan
@@ -1,8 +1,7 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-04.plan
index 7cd72c2..e981647 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-04.plan
@@ -1,8 +1,7 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-05.plan
index 7cd72c2..e981647 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-05.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-05.plan
@@ -1,8 +1,7 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-06.plan
index 7cd72c2..e981647 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-06.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-06.plan
@@ -1,8 +1,7 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-07.plan
index 7cd72c2..e981647 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-07.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-geometry/skip-rtree-sidx-07.plan
@@ -1,8 +1,7 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
index 582253a..6323247 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
@@ -1,54 +1,29 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$61(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$94]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- STREAM_SELECT  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$94(ASC), $$55(ASC)]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$94]  |PARTITIONED|
-                              -- UNION_ALL  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- STREAM_SELECT  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- SPLIT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
-                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- ASSIGN  |PARTITIONED|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                      -- ASSIGN  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- STREAM_SELECT  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$64(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$64(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$97]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- STREAM_SELECT  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$97(ASC), $$58(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$97]  |PARTITIONED|
+                            -- UNION_ALL  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                               -- SPLIT  |PARTITIONED|
@@ -59,61 +34,55 @@
                                                         -- BROADCAST_EXCHANGE  |PARTITIONED|
                                                           -- ASSIGN  |PARTITIONED|
                                                             -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- ASSIGN  |PARTITIONED|
-                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$94]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- STREAM_SELECT  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$94(ASC), $$55(ASC)]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$94]  |PARTITIONED|
-                                        -- UNION_ALL  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- SPLIT  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
-                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- ASSIGN  |PARTITIONED|
-                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- SPLIT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- STREAM_PROJECT  |PARTITIONED|
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$97]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$97(ASC), $$58(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$97]  |PARTITIONED|
+                                      -- UNION_ALL  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                         -- SPLIT  |PARTITIONED|
@@ -124,10 +93,28 @@
                                                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                                                     -- ASSIGN  |PARTITIONED|
                                                                       -- ASSIGN  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- SPLIT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
+                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                           -- ASSIGN  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.plan
index b6e4dbc..d39aee0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.plan
@@ -1,37 +1,33 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$66(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$58]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- STREAM_SELECT  |LOCAL|
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$58(ASC), $$59(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$78(ASC)]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$69(ASC) ]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$61]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- STREAM_SELECT  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$61(ASC), $$62(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$61]  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$81(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                     -- ASSIGN  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
index e183f09..4b2546a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
@@ -1,83 +1,76 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$66(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$66(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$58]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- STREAM_SELECT  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$58(ASC), $$59(ASC)]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STABLE_SORT [$$78(ASC)]  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
-                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- ASSIGN  |PARTITIONED|
-                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$58]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- STREAM_SELECT  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$69(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$69(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$61]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- STREAM_SELECT  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$61(ASC), $$62(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$61]  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$58(ASC), $$59(ASC)]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- STREAM_SELECT  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$81(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- STABLE_SORT [$$78(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
-                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                    -- ASSIGN  |PARTITIONED|
-                                                                      -- ASSIGN  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$61]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$61(ASC), $$62(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$61]  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STABLE_SORT [$$81(ASC)]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- RTREE_SEARCH (test.TweetMessages.twmSndLocIx)  |PARTITIONED|
+                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                           -- ASSIGN  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- BTREE_SEARCH (test.TweetMessages.TweetMessages)  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_02.plan
index c992324..62ab9c9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_02.plan
@@ -1,23 +1,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.MyData2.MyData2)  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- BTREE_SEARCH (test.MyData2.MyData2)  |PARTITIONED|
+                -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$40(ASC)]  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- RTREE_SEARCH (test.MyData2.rtree_index)  |PARTITIONED|
-                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
+                        -- RTREE_SEARCH (test.MyData2.rtree_index)  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test.MyData1)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (test.MyData1)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_03.plan
index e5d716f..cc41707 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_03.plan
@@ -1,23 +1,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.MyData.MyData)  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- BTREE_SEARCH (test.MyData.MyData)  |PARTITIONED|
+                -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$40(ASC)]  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- RTREE_SEARCH (test.MyData.rtree_index)  |PARTITIONED|
-                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
+                        -- RTREE_SEARCH (test.MyData.rtree_index)  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test.MyData)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (test.MyData)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-sidx-idxonly-01-disable-idxonly.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-sidx-idxonly-01-disable-idxonly.plan
index 50ddb2c..7d4f008 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-sidx-idxonly-01-disable-idxonly.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-sidx-idxonly-01-disable-idxonly.plan
@@ -1,20 +1,19 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH (test.Fragile_raw.Fragile_raw)  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STABLE_SORT [$$63(ASC)]  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- RTREE_SEARCH (test.Fragile_raw.cfLocation)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (test.Fragile_raw.Fragile_raw)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$66(ASC)]  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- RTREE_SEARCH (test.Fragile_raw.cfLocation)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-sidx-idxonly-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-sidx-idxonly-01.plan
index a87edbc..36f14dd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-sidx-idxonly-01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-sidx-idxonly-01.plan
@@ -1,31 +1,16 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- UNION_ALL  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- UNION_ALL  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.Fragile_raw.Fragile_raw)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- SPLIT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- RTREE_SEARCH (test.Fragile_raw.cfLocation)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.Fragile_raw.Fragile_raw)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- SPLIT  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -35,3 +20,15 @@
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- ASSIGN  |PARTITIONED|
                                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- SPLIT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- RTREE_SEARCH (test.Fragile_raw.cfLocation)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-secondary-btree-index-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-secondary-btree-index-2.plan
index 74f6ebd..15424cd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-secondary-btree-index-2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-secondary-btree-index-2.plan
@@ -1,17 +1,16 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- BTREE_SEARCH (test.testdst.testdst)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.testdst.sec_Idx)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.testdst.testdst)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.testdst.sec_Idx)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-secondary-btree-index-4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-secondary-btree-index-4.plan
new file mode 100644
index 0000000..1435e63
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-secondary-btree-index-4.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/spatial_intersect_dynamic_partitioning.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/spatial_intersect_dynamic_partitioning.plan
index ce4f736..423affb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/spatial_intersect_dynamic_partitioning.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/spatial_intersect_dynamic_partitioning.plan
@@ -1,156 +1,140 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- NESTED_LOOP  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- SPATIAL_JOIN [$$60, $$49] [$$61, $$50]  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STABLE_SORT [$$60(ASC), $$49(ASC)]  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$60]  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- UNNEST  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- SPATIAL_JOIN [$$63, $$52] [$$64, $$53]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$63(ASC), $$52(ASC)]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$63]  |PARTITIONED|
+                          -- UNNEST  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- NESTED_LOOP  |PARTITIONED|
+                                -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                                  -- REPLICATE  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN (test.ParkSet)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- NESTED_LOOP  |PARTITIONED|
-                                    -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
                                       -- REPLICATE  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- DATASOURCE_SCAN (test.ParkSet)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                              -- NESTED_LOOP  |UNPARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                  -- AGGREGATE  |UNPARTITIONED|
+                                                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                      -- AGGREGATE  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- REPLICATE  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- DATASOURCE_SCAN (test.ParkSet)  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                  -- AGGREGATE  |UNPARTITIONED|
+                                                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                      -- AGGREGATE  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- REPLICATE  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- DATASOURCE_SCAN (test.LakeSet)  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$64(ASC), $$53(ASC)]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$64]  |PARTITIONED|
+                          -- UNNEST  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- NESTED_LOOP  |PARTITIONED|
+                                -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                                  -- REPLICATE  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN (test.LakeSet)  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                      -- NESTED_LOOP  |UNPARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                          -- AGGREGATE  |UNPARTITIONED|
-                                                            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                              -- AGGREGATE  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- REPLICATE  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        -- ASSIGN  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- DATASOURCE_SCAN (test.ParkSet)  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                          -- AGGREGATE  |UNPARTITIONED|
-                                                            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                              -- AGGREGATE  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- REPLICATE  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        -- ASSIGN  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- DATASOURCE_SCAN (test.LakeSet)  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STABLE_SORT [$$61(ASC), $$50(ASC)]  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$61]  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- UNNEST  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- NESTED_LOOP  |PARTITIONED|
-                                    -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
                                       -- REPLICATE  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- DATASOURCE_SCAN (test.LakeSet)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                      -- NESTED_LOOP  |UNPARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                          -- AGGREGATE  |UNPARTITIONED|
-                                                            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                              -- AGGREGATE  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- REPLICATE  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        -- ASSIGN  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- DATASOURCE_SCAN (test.ParkSet)  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                          -- AGGREGATE  |UNPARTITIONED|
-                                                            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                              -- AGGREGATE  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- REPLICATE  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        -- ASSIGN  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- DATASOURCE_SCAN (test.LakeSet)  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- REPLICATE  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                              -- NESTED_LOOP  |UNPARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                  -- AGGREGATE  |UNPARTITIONED|
+                                                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                      -- AGGREGATE  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- REPLICATE  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- DATASOURCE_SCAN (test.ParkSet)  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                  -- AGGREGATE  |UNPARTITIONED|
+                                                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                      -- AGGREGATE  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- REPLICATE  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- DATASOURCE_SCAN (test.LakeSet)  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                          -- NESTED_LOOP  |UNPARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                              -- NESTED_LOOP  |UNPARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                  -- AGGREGATE  |UNPARTITIONED|
-                                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                      -- AGGREGATE  |PARTITIONED|
+                              -- AGGREGATE  |UNPARTITIONED|
+                                -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                  -- AGGREGATE  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- REPLICATE  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- REPLICATE  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (test.ParkSet)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                  -- AGGREGATE  |UNPARTITIONED|
-                                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                      -- AGGREGATE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.ParkSet)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                              -- AGGREGATE  |UNPARTITIONED|
+                                -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                  -- AGGREGATE  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- REPLICATE  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- REPLICATE  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (test.LakeSet)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.LakeSet)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/spatial_intersect_static_partitioning.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/spatial_intersect_static_partitioning.plan
index ce2f6e6..dff157c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/spatial_intersect_static_partitioning.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/spatial_intersect_static_partitioning.plan
@@ -1,32 +1,29 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- SPATIAL_JOIN [$$53, $$49] [$$54, $$50]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$53(ASC), $$49(ASC)]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
-                          -- UNNEST  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- SPATIAL_JOIN [$$56, $$52] [$$57, $$53]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$56(ASC), $$52(ASC)]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
+                        -- UNNEST  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
                             -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN (test.ParkSet)  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (test.ParkSet)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$54(ASC), $$50(ASC)]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
-                          -- UNNEST  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$57(ASC), $$53(ASC)]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$57]  |PARTITIONED|
+                        -- UNNEST  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
                             -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN (test.LakeSet)  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (test.LakeSet)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_distance_static_partitioning.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_distance_static_partitioning.plan
index f66785b..a4b7aba 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_distance_static_partitioning.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_distance_static_partitioning.plan
@@ -1,34 +1,31 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- SPATIAL_JOIN [$$62, $$56] [$$63, $$57]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$62(ASC), $$56(ASC)]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
-                          -- UNNEST  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- SPATIAL_JOIN [$$65, $$59] [$$66, $$60]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$65(ASC), $$59(ASC)]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$65]  |PARTITIONED|
+                        -- UNNEST  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.ParkSet)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.ParkSet)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$63(ASC), $$57(ASC)]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$63]  |PARTITIONED|
-                          -- UNNEST  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$66(ASC), $$60(ASC)]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$66]  |PARTITIONED|
+                        -- UNNEST  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.LakeSet)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.LakeSet)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_intersects_static_partitioning.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_intersects_static_partitioning.plan
index 43d8da6..fae5476 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_intersects_static_partitioning.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_intersects_static_partitioning.plan
@@ -1,34 +1,31 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- SPATIAL_JOIN [$$55, $$52] [$$56, $$53]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$55(ASC), $$52(ASC)]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$55]  |PARTITIONED|
-                          -- UNNEST  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- SPATIAL_JOIN [$$58, $$55] [$$59, $$56]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$58(ASC), $$55(ASC)]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
+                        -- UNNEST  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.ParkSet)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.ParkSet)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$56(ASC), $$53(ASC)]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
-                          -- UNNEST  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$59(ASC), $$56(ASC)]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$59]  |PARTITIONED|
+                        -- UNNEST  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.LakeSet)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.LakeSet)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization.plan
index 56d5533..6649f67 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization.plan
@@ -1,31 +1,27 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$91(ASC), $$92(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$91(ASC), $$92(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- NESTED_LOOP  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$97(ASC), $$98(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$97(ASC), $$98(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- NESTED_LOOP  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- REPLICATE  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- STREAM_SELECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (TinySocial.FacebookUsers)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
                         -- STREAM_SELECT  |PARTITIONED|
                           -- ASSIGN  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- DATASOURCE_SCAN (TinySocial.FacebookUsers)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (TinySocial.FacebookUsers)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization_ps.plan
index 8ba2609..979e707 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization_ps.plan
@@ -1,67 +1,60 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$91(ASC), $$92(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$91(ASC), $$92(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- NESTED_LOOP  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$97(ASC), $$98(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$97(ASC), $$98(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- NESTED_LOOP  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- REPLICATE  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- STREAM_SELECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (TinySocial.FacebookUsers)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- STREAM_SELECT  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- DATASOURCE_SCAN (TinySocial.FacebookUsers)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- NESTED_LOOP  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (TinySocial.FacebookUsers)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- NESTED_LOOP  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- REPLICATE  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- REPLICATE  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- STREAM_SELECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (TinySocial.FacebookUsers)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                    -- REPLICATE  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
                                           -- STREAM_SELECT  |PARTITIONED|
                                             -- ASSIGN  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                 -- DATASOURCE_SCAN (TinySocial.FacebookUsers)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- REPLICATE  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN (TinySocial.FacebookUsers)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-01.plan
index e800b4e..b87eeb5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-01.plan
@@ -1,17 +1,15 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+              -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+                      -- BTREE_SEARCH (test.TestOpen.idx_s)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.TestOpen.idx_s)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-02.plan
index e800b4e..b87eeb5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-02.plan
@@ -1,17 +1,15 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+              -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+                      -- BTREE_SEARCH (test.TestOpen.idx_s)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.TestOpen.idx_s)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-03.plan
index e800b4e..b87eeb5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-03.plan
@@ -1,17 +1,15 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- BTREE_SEARCH (test.TestOpen.TestOpen)  |PARTITIONED|
+              -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+                      -- BTREE_SEARCH (test.TestOpen.idx_s)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH (test.TestOpen.idx_s)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |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 d72e195..164bc8c 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
@@ -1,21 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$30(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (test.employee.employee)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (test.employee.employee)  |PARTITIONED|
+                      -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- BTREE_SEARCH (test.employee.idx_employee_fname)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.employee.idx_employee_fname)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |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 1697959..c044320 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
@@ -1,32 +1,30 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$31(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (test.employee.employee)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (test.employee.employee)  |PARTITIONED|
+                      -- INTERSECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- INTERSECT  |PARTITIONED|
+                          -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- BTREE_SEARCH (test.employee.idx_employee_fname)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (test.employee.idx_employee_fname)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- BTREE_SEARCH (test.employee.idx_employee_lname)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (test.employee.idx_employee_lname)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists.plan
index b81c831..f738e90 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists.plan
@@ -1,74 +1,68 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$cntrycode(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$187]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$187]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$162]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- SORT_GROUP_BY[$$184]  |PARTITIONED|
-                                      {
-                                        -- AGGREGATE  |LOCAL|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$cntrycode(ASC) ]  |PARTITIONED|
+        -- SORT_GROUP_BY[$$196]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$196]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$171]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$193]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- HASH_PARTITION_EXCHANGE [$$193]  |PARTITIONED|
+                            -- PRE_CLUSTERED_GROUP_BY[$$187]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
                                           -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                      }
-                                -- HASH_PARTITION_EXCHANGE [$$184]  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$178]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$178(ASC)]  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$174][$$171]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$174]  |PARTITIONED|
-                                                  -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- NESTED_LOOP  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (test.Customer)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |UNPARTITIONED|
-                                                              -- ASSIGN  |UNPARTITIONED|
-                                                                -- AGGREGATE  |UNPARTITIONED|
-                                                                  -- AGGREGATE  |UNPARTITIONED|
-                                                                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                                      -- AGGREGATE  |PARTITIONED|
-                                                                        -- STREAM_SELECT  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- DATASOURCE_SCAN (test.Customer)  |PARTITIONED|
-                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$171]  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$187(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$183][$$180]  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$183]  |PARTITIONED|
+                                            -- RUNNING_AGGREGATE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- NESTED_LOOP  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (test.Orders)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (test.Customer)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                      -- ASSIGN  |UNPARTITIONED|
+                                                        -- AGGREGATE  |UNPARTITIONED|
+                                                          -- AGGREGATE  |UNPARTITIONED|
+                                                            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                              -- AGGREGATE  |PARTITIONED|
+                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- DATASOURCE_SCAN (test.Customer)  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$180]  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |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/exists_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists_ps.plan
index 2cccba4..274bc84 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists_ps.plan
@@ -1,157 +1,146 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$cntrycode(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$cntrycode(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- SORT_GROUP_BY[$$187]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- HASH_PARTITION_EXCHANGE [$$187]  |PARTITIONED|
-                          -- SORT_GROUP_BY[$$162]  |PARTITIONED|
-                                  {
-                                    -- AGGREGATE  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                  }
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- STREAM_SELECT  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- SORT_GROUP_BY[$$184]  |PARTITIONED|
-                                                  {
-                                                    -- AGGREGATE  |LOCAL|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$cntrycode(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$cntrycode(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- SORT_GROUP_BY[$$196]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- HASH_PARTITION_EXCHANGE [$$196]  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$171]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$193]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- HASH_PARTITION_EXCHANGE [$$193]  |PARTITIONED|
+                                        -- PRE_CLUSTERED_GROUP_BY[$$187]  |PARTITIONED|
+                                                {
+                                                  -- AGGREGATE  |LOCAL|
+                                                    -- STREAM_SELECT  |LOCAL|
                                                       -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                  }
-                                            -- HASH_PARTITION_EXCHANGE [$$184]  |PARTITIONED|
-                                              -- PRE_CLUSTERED_GROUP_BY[$$178]  |PARTITIONED|
+                                                }
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$187(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$183][$$180]  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$183]  |PARTITIONED|
+                                                        -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- NESTED_LOOP  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- DATASOURCE_SCAN (test.Customer)  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |UNPARTITIONED|
+                                                                    -- AGGREGATE  |UNPARTITIONED|
+                                                                      -- AGGREGATE  |UNPARTITIONED|
+                                                                        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                                          -- AGGREGATE  |PARTITIONED|
+                                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- DATASOURCE_SCAN (test.Customer)  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$180]  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (test.Orders)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- SORT_GROUP_BY[$$196]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- HASH_PARTITION_EXCHANGE [$$196]  |PARTITIONED|
+                                  -- SORT_GROUP_BY[$$171]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- SORT_GROUP_BY[$$193]  |PARTITIONED|
                                                       {
                                                         -- AGGREGATE  |LOCAL|
-                                                          -- STREAM_SELECT  |LOCAL|
-                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                       }
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STABLE_SORT [$$178(ASC)]  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$193]  |PARTITIONED|
+                                                  -- PRE_CLUSTERED_GROUP_BY[$$187]  |PARTITIONED|
+                                                          {
+                                                            -- AGGREGATE  |LOCAL|
+                                                              -- STREAM_SELECT  |LOCAL|
+                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                          }
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- STABLE_SORT [$$187(ASC)]  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- HYBRID_HASH_JOIN [$$174][$$171]  |PARTITIONED|
-                                                            -- HASH_PARTITION_EXCHANGE [$$174]  |PARTITIONED|
-                                                              -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- NESTED_LOOP  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- HYBRID_HASH_JOIN [$$183][$$180]  |PARTITIONED|
+                                                                -- HASH_PARTITION_EXCHANGE [$$183]  |PARTITIONED|
+                                                                  -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                                    -- STREAM_PROJECT  |PARTITIONED|
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- ASSIGN  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- NESTED_LOOP  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                             -- ASSIGN  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- DATASOURCE_SCAN (test.Customer)  |PARTITIONED|
-                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |UNPARTITIONED|
-                                                                          -- ASSIGN  |UNPARTITIONED|
-                                                                            -- AGGREGATE  |UNPARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- DATASOURCE_SCAN (test.Customer)  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                            -- ASSIGN  |UNPARTITIONED|
                                                                               -- AGGREGATE  |UNPARTITIONED|
-                                                                                -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                                                  -- AGGREGATE  |PARTITIONED|
-                                                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                -- AGGREGATE  |UNPARTITIONED|
+                                                                                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                                                    -- AGGREGATE  |PARTITIONED|
+                                                                                      -- STREAM_SELECT  |PARTITIONED|
                                                                                         -- ASSIGN  |PARTITIONED|
                                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                             -- DATASOURCE_SCAN (test.Customer)  |PARTITIONED|
                                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                            -- HASH_PARTITION_EXCHANGE [$$171]  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (test.Orders)  |PARTITIONED|
+                                                                -- HASH_PARTITION_EXCHANGE [$$180]  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- SORT_GROUP_BY[$$187]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- HASH_PARTITION_EXCHANGE [$$187]  |PARTITIONED|
-                                    -- SORT_GROUP_BY[$$162]  |PARTITIONED|
-                                            {
-                                              -- AGGREGATE  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                            }
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- SORT_GROUP_BY[$$184]  |PARTITIONED|
-                                                            {
-                                                              -- AGGREGATE  |LOCAL|
-                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                            }
-                                                      -- HASH_PARTITION_EXCHANGE [$$184]  |PARTITIONED|
-                                                        -- PRE_CLUSTERED_GROUP_BY[$$178]  |PARTITIONED|
-                                                                {
-                                                                  -- AGGREGATE  |LOCAL|
-                                                                    -- STREAM_SELECT  |LOCAL|
-                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                }
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- STABLE_SORT [$$178(ASC)]  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- HYBRID_HASH_JOIN [$$174][$$171]  |PARTITIONED|
-                                                                      -- HASH_PARTITION_EXCHANGE [$$174]  |PARTITIONED|
-                                                                        -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- NESTED_LOOP  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                          -- DATASOURCE_SCAN (test.Customer)  |PARTITIONED|
-                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                  -- STREAM_PROJECT  |UNPARTITIONED|
-                                                                                    -- ASSIGN  |UNPARTITIONED|
-                                                                                      -- AGGREGATE  |UNPARTITIONED|
-                                                                                        -- AGGREGATE  |UNPARTITIONED|
-                                                                                          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                                                            -- AGGREGATE  |PARTITIONED|
-                                                                                              -- STREAM_SELECT  |PARTITIONED|
-                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                      -- DATASOURCE_SCAN (test.Customer)  |PARTITIONED|
-                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                      -- HASH_PARTITION_EXCHANGE [$$171]  |PARTITIONED|
-                                                                        -- ASSIGN  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- DATASOURCE_SCAN (test.Orders)  |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.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in.plan
index 79e304c..23197c9 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
@@ -1,35 +1,31 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$53(ASC) ]  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$53(ASC) ]  |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[$$48]  |PARTITIONED|
-                        {
-                          -- AGGREGATE  |LOCAL|
-                            -- STREAM_SELECT  |LOCAL|
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                        }
+                -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$48][$$37]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$48][$$37]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$48]  |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|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.Orders)  |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|
-                                        -- DATASOURCE_SCAN (test.Orders)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_1.plan
index fd9c4a4..0afd76c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_1.plan
@@ -1,17 +1,16 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$15][$$21]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$16][$$22]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
               -- BROADCAST_EXCHANGE  |PARTITIONED|
                 -- UNNEST  |UNPARTITIONED|
                   -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_1_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_1_ps.plan
index 0eb2166..d3629e5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_1_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_1_ps.plan
@@ -1,21 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-      -- RANGE_PARTITION_EXCHANGE [$$20(ASC)]  |PARTITIONED|
+    -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+      -- RANGE_PARTITION_EXCHANGE [$$21(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 [$$15][$$21]  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$16][$$22]  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                         -- UNNEST  |UNPARTITIONED|
                           -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
@@ -28,14 +27,13 @@
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$15][$$21]  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$16][$$22]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                               -- BROADCAST_EXCHANGE  |PARTITIONED|
                                 -- UNNEST  |UNPARTITIONED|
                                   -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_2.plan
index 22ea58a..ed5067c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_2.plan
@@ -1,28 +1,26 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$21]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- STREAM_SELECT  |LOCAL|
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$22][$#1]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$26(ASC) ]  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$22]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- STREAM_SELECT  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$23][$#1]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$23]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
-                          -- ASSIGN  |UNPARTITIONED|
-                            -- UNNEST  |UNPARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                        -- ASSIGN  |UNPARTITIONED|
+                          -- UNNEST  |UNPARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_2_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_2_ps.plan
index 84a9315..d587010 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_2_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_2_ps.plan
@@ -1,37 +1,35 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
-      -- RANGE_PARTITION_EXCHANGE [$$25(ASC)]  |PARTITIONED|
+    -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+      -- RANGE_PARTITION_EXCHANGE [$$26(ASC)]  |PARTITIONED|
         -- FORWARD  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- REPLICATE  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$21]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- STREAM_SELECT  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$22][$#1]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$22]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- STREAM_SELECT  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$23][$#1]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$23]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
-                                      -- ASSIGN  |UNPARTITIONED|
-                                        -- UNNEST  |UNPARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                                    -- ASSIGN  |UNPARTITIONED|
+                                      -- UNNEST  |UNPARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
           -- BROADCAST_EXCHANGE  |PARTITIONED|
             -- AGGREGATE  |UNPARTITIONED|
               -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
@@ -39,29 +37,27 @@
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- REPLICATE  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- STREAM_SELECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_CLUSTERED_GROUP_BY[$$21]  |PARTITIONED|
-                                      {
-                                        -- AGGREGATE  |LOCAL|
-                                          -- STREAM_SELECT  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                      }
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- HYBRID_HASH_JOIN [$$22][$#1]  |PARTITIONED|
-                                            -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_CLUSTERED_GROUP_BY[$$22]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$23][$#1]  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$23]  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                            -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
-                                              -- ASSIGN  |UNPARTITIONED|
-                                                -- UNNEST  |UNPARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                                            -- ASSIGN  |UNPARTITIONED|
+                                              -- UNNEST  |UNPARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_3.plan
index fd9c4a4..0afd76c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_3.plan
@@ -1,17 +1,16 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$15][$$21]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$16][$$22]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
               -- BROADCAST_EXCHANGE  |PARTITIONED|
                 -- UNNEST  |UNPARTITIONED|
                   -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_3_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_3_ps.plan
index 0eb2166..d3629e5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_3_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_3_ps.plan
@@ -1,21 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-      -- RANGE_PARTITION_EXCHANGE [$$20(ASC)]  |PARTITIONED|
+    -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+      -- RANGE_PARTITION_EXCHANGE [$$21(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 [$$15][$$21]  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$16][$$22]  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                         -- UNNEST  |UNPARTITIONED|
                           -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
@@ -28,14 +27,13 @@
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$15][$$21]  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$16][$$22]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                               -- BROADCAST_EXCHANGE  |PARTITIONED|
                                 -- UNNEST  |UNPARTITIONED|
                                   -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_4.plan
index 5815097..0f33a1e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_4.plan
@@ -1,28 +1,26 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$20]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- STREAM_SELECT  |LOCAL|
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$21][$#1]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$21]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- STREAM_SELECT  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$22][$#1]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
-                          -- ASSIGN  |UNPARTITIONED|
-                            -- UNNEST  |UNPARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                        -- ASSIGN  |UNPARTITIONED|
+                          -- UNNEST  |UNPARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_4_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_4_ps.plan
index f7af324..55d6709 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_4_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_4_ps.plan
@@ -1,37 +1,35 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-      -- RANGE_PARTITION_EXCHANGE [$$24(ASC)]  |PARTITIONED|
+    -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+      -- RANGE_PARTITION_EXCHANGE [$$25(ASC)]  |PARTITIONED|
         -- FORWARD  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- REPLICATE  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$20]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- STREAM_SELECT  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$21][$#1]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$21]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- STREAM_SELECT  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$22][$#1]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
-                                      -- ASSIGN  |UNPARTITIONED|
-                                        -- UNNEST  |UNPARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                                    -- ASSIGN  |UNPARTITIONED|
+                                      -- UNNEST  |UNPARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
           -- BROADCAST_EXCHANGE  |PARTITIONED|
             -- AGGREGATE  |UNPARTITIONED|
               -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
@@ -39,29 +37,27 @@
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- REPLICATE  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- STREAM_SELECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_CLUSTERED_GROUP_BY[$$20]  |PARTITIONED|
-                                      {
-                                        -- AGGREGATE  |LOCAL|
-                                          -- STREAM_SELECT  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                      }
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- HYBRID_HASH_JOIN [$$21][$#1]  |PARTITIONED|
-                                            -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_CLUSTERED_GROUP_BY[$$21]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$22][$#1]  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                            -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
-                                              -- ASSIGN  |UNPARTITIONED|
-                                                -- UNNEST  |UNPARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                                            -- ASSIGN  |UNPARTITIONED|
+                                              -- UNNEST  |UNPARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_5.plan
index fd9c4a4..0afd76c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_5.plan
@@ -1,17 +1,16 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$15][$$21]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$16][$$22]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
               -- BROADCAST_EXCHANGE  |PARTITIONED|
                 -- UNNEST  |UNPARTITIONED|
                   -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_5_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_5_ps.plan
index 0eb2166..d3629e5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_5_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_5_ps.plan
@@ -1,21 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-      -- RANGE_PARTITION_EXCHANGE [$$20(ASC)]  |PARTITIONED|
+    -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+      -- RANGE_PARTITION_EXCHANGE [$$21(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 [$$15][$$21]  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$16][$$22]  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                         -- UNNEST  |UNPARTITIONED|
                           -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
@@ -28,14 +27,13 @@
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$15][$$21]  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$16][$$22]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                               -- BROADCAST_EXCHANGE  |PARTITIONED|
                                 -- UNNEST  |UNPARTITIONED|
                                   -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_6.plan
index 22ea58a..ed5067c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_6.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_6.plan
@@ -1,28 +1,26 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$21]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- STREAM_SELECT  |LOCAL|
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$22][$#1]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$26(ASC) ]  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$22]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- STREAM_SELECT  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$23][$#1]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$23]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
-                          -- ASSIGN  |UNPARTITIONED|
-                            -- UNNEST  |UNPARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                        -- ASSIGN  |UNPARTITIONED|
+                          -- UNNEST  |UNPARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_6_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_6_ps.plan
index 84a9315..d587010 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_6_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_6_ps.plan
@@ -1,37 +1,35 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
-      -- RANGE_PARTITION_EXCHANGE [$$25(ASC)]  |PARTITIONED|
+    -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+      -- RANGE_PARTITION_EXCHANGE [$$26(ASC)]  |PARTITIONED|
         -- FORWARD  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- REPLICATE  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$21]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- STREAM_SELECT  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$22][$#1]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$22]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- STREAM_SELECT  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$23][$#1]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$23]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
-                                      -- ASSIGN  |UNPARTITIONED|
-                                        -- UNNEST  |UNPARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                                    -- ASSIGN  |UNPARTITIONED|
+                                      -- UNNEST  |UNPARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
           -- BROADCAST_EXCHANGE  |PARTITIONED|
             -- AGGREGATE  |UNPARTITIONED|
               -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
@@ -39,29 +37,27 @@
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- REPLICATE  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- STREAM_SELECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_CLUSTERED_GROUP_BY[$$21]  |PARTITIONED|
-                                      {
-                                        -- AGGREGATE  |LOCAL|
-                                          -- STREAM_SELECT  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                      }
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- HYBRID_HASH_JOIN [$$22][$#1]  |PARTITIONED|
-                                            -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_CLUSTERED_GROUP_BY[$$22]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$23][$#1]  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$23]  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                            -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
-                                              -- ASSIGN  |UNPARTITIONED|
-                                                -- UNNEST  |UNPARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                                            -- ASSIGN  |UNPARTITIONED|
+                                              -- UNNEST  |UNPARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
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 bcdb951..c865744 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
@@ -1,33 +1,29 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$58(ASC) ]  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$58(ASC) ]  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- PRE_CLUSTERED_GROUP_BY[$$52]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- STREAM_SELECT  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- PRE_CLUSTERED_GROUP_BY[$$52]  |PARTITIONED|
-                        {
-                          -- AGGREGATE  |LOCAL|
-                            -- STREAM_SELECT  |LOCAL|
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                        }
+                -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- HYBRID_HASH_JOIN [$$52][$$51]  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- STREAM_SELECT  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$52][$$51]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$52]  |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 [$$51]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.Orders)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$51]  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (test.Orders)  |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_correlated_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated_ps.plan
index 9cb3421..e53c8b9 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
@@ -1,74 +1,67 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$58(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$58(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_CLUSTERED_GROUP_BY[$$52]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- PRE_CLUSTERED_GROUP_BY[$$52]  |PARTITIONED|
-                                    {
-                                      -- AGGREGATE  |LOCAL|
-                                        -- STREAM_SELECT  |LOCAL|
-                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                    }
+                            -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- HYBRID_HASH_JOIN [$$52][$$51]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- STREAM_SELECT  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$52][$$51]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$52]  |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 [$$51]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.Orders)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$51]  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$52]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$52][$$51]  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
+                                            -- STREAM_SELECT  |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|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- PRE_CLUSTERED_GROUP_BY[$$52]  |PARTITIONED|
-                                            {
-                                              -- AGGREGATE  |LOCAL|
-                                                -- STREAM_SELECT  |LOCAL|
-                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                            }
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- HYBRID_HASH_JOIN [$$52][$$51]  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- STREAM_SELECT  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$51]  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (test.Orders)  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (test.Customers)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$51]  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (test.Orders)  |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_let_3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_let_3.plan
index ec77d62..2187811 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_let_3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_let_3.plan
@@ -1,74 +1,64 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$101(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$101(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$107(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$107(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
               -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- PRE_CLUSTERED_GROUP_BY[$$110]  |PARTITIONED|
+                          {
+                            -- AGGREGATE  |LOCAL|
+                              -- STREAM_SELECT  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                          }
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$104]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- STREAM_SELECT  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$104(ASC)]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$104]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$97][$$pid]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$97]  |PARTITIONED|
-                                      -- STREAM_SELECT  |PARTITIONED|
-                                        -- RUNNING_AGGREGATE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- UNNEST  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ASSIGN  |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 (test.cart)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$110(ASC)]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$110]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$103][$$pid]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$103]  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- RUNNING_AGGREGATE  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- STREAM_SELECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- SORT_GROUP_BY[$$108]  |PARTITIONED|
-                                                      {
-                                                        -- AGGREGATE  |LOCAL|
-                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                      }
-                                                -- HASH_PARTITION_EXCHANGE [$$108]  |PARTITIONED|
-                                                  -- SORT_GROUP_BY[$$92]  |PARTITIONED|
-                                                          {
-                                                            -- AGGREGATE  |LOCAL|
-                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                          }
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- STREAM_SELECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- UNNEST  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ASSIGN  |PARTITIONED|
+                                        -- UNNEST  |PARTITIONED|
+                                          -- ASSIGN  |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 (test.cart)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- SORT_GROUP_BY[$$114]  |PARTITIONED|
+                                                {
+                                                  -- AGGREGATE  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                }
+                                          -- HASH_PARTITION_EXCHANGE [$$114]  |PARTITIONED|
+                                            -- SORT_GROUP_BY[$$98]  |PARTITIONED|
+                                                    {
+                                                      -- AGGREGATE  |LOCAL|
+                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                    }
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_SELECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- UNNEST  |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 (test.cart)  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- REPLICATE  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- 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_let_4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_let_4.plan
index ee9243d..ce56876 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_let_4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_let_4.plan
@@ -1,74 +1,64 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$120(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$120(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$127(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$127(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
               -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- PRE_CLUSTERED_GROUP_BY[$$130]  |PARTITIONED|
+                          {
+                            -- AGGREGATE  |LOCAL|
+                              -- STREAM_SELECT  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                          }
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$123]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- STREAM_SELECT  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$123(ASC)]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$123]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$116][$$108]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$116]  |PARTITIONED|
-                                      -- STREAM_SELECT  |PARTITIONED|
-                                        -- RUNNING_AGGREGATE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- UNNEST  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ASSIGN  |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 (test.cart)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$130(ASC)]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$123][$$115]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$123]  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- RUNNING_AGGREGATE  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- STREAM_SELECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- SORT_GROUP_BY[$$127]  |PARTITIONED|
-                                                      {
-                                                        -- AGGREGATE  |LOCAL|
-                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                      }
-                                                -- HASH_PARTITION_EXCHANGE [$$127]  |PARTITIONED|
-                                                  -- SORT_GROUP_BY[$$110]  |PARTITIONED|
-                                                          {
-                                                            -- AGGREGATE  |LOCAL|
-                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                          }
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- STREAM_SELECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- UNNEST  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ASSIGN  |PARTITIONED|
+                                        -- UNNEST  |PARTITIONED|
+                                          -- ASSIGN  |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 (test.cart)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- SORT_GROUP_BY[$$134]  |PARTITIONED|
+                                                {
+                                                  -- AGGREGATE  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                }
+                                          -- HASH_PARTITION_EXCHANGE [$$134]  |PARTITIONED|
+                                            -- SORT_GROUP_BY[$$117]  |PARTITIONED|
+                                                    {
+                                                      -- AGGREGATE  |LOCAL|
+                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                    }
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_SELECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- UNNEST  |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 (test.cart)  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- REPLICATE  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- DATASOURCE_SCAN (test.cart)  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_let_5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_let_5.plan
index c670194..ecaefdf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_let_5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_let_5.plan
@@ -1,74 +1,64 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$120(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$120(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$127(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$127(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
               -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- PRE_CLUSTERED_GROUP_BY[$$130]  |PARTITIONED|
+                          {
+                            -- AGGREGATE  |LOCAL|
+                              -- STREAM_SELECT  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                          }
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$123]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- STREAM_SELECT  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$123(ASC)]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$123]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                      -- STABLE_SORT [$$130(ASC)]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$123][$$115]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$116][$$108]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_SELECT  |PARTITIONED|
-                                        -- RUNNING_AGGREGATE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- UNNEST  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ASSIGN  |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 (test.cart)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- RUNNING_AGGREGATE  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- STREAM_SELECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- SORT_GROUP_BY[$$127]  |PARTITIONED|
-                                                      {
-                                                        -- AGGREGATE  |LOCAL|
-                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                      }
-                                                -- HASH_PARTITION_EXCHANGE [$$127]  |PARTITIONED|
-                                                  -- SORT_GROUP_BY[$$110]  |PARTITIONED|
-                                                          {
-                                                            -- AGGREGATE  |LOCAL|
-                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                          }
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- STREAM_SELECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- UNNEST  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ASSIGN  |PARTITIONED|
+                                        -- UNNEST  |PARTITIONED|
+                                          -- ASSIGN  |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 (test.cart)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- SORT_GROUP_BY[$$134]  |PARTITIONED|
+                                                {
+                                                  -- AGGREGATE  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                }
+                                          -- HASH_PARTITION_EXCHANGE [$$134]  |PARTITIONED|
+                                            -- SORT_GROUP_BY[$$117]  |PARTITIONED|
+                                                    {
+                                                      -- AGGREGATE  |LOCAL|
+                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                    }
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_SELECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- UNNEST  |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 (test.cart)  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- REPLICATE  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- DATASOURCE_SCAN (test.cart)  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_let_6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_let_6.plan
index 512a0e7..3febf26 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_let_6.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_let_6.plan
@@ -1,70 +1,60 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$111(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$111(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$118(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$118(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
               -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- SUBPLAN  |PARTITIONED|
-                            {
-                              -- AGGREGATE  |LOCAL|
-                                -- STREAM_SELECT  |LOCAL|
-                                  -- UNNEST  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                            }
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- UNNEST  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- NESTED_LOOP  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- NESTED_LOOP  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- UNNEST  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
                                 -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- UNNEST  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |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 (test.cart)  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- REPLICATE  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (test.cart)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- AGGREGATE  |UNPARTITIONED|
-                              -- SORT_MERGE_EXCHANGE [$$pid(ASC) ]  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- SORT_GROUP_BY[$$117]  |PARTITIONED|
-                                              {
-                                                -- AGGREGATE  |LOCAL|
-                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                              }
-                                        -- HASH_PARTITION_EXCHANGE [$$117]  |PARTITIONED|
-                                          -- SORT_GROUP_BY[$$102]  |PARTITIONED|
-                                                  {
-                                                    -- AGGREGATE  |LOCAL|
-                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                  }
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- STREAM_SELECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- UNNEST  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |UNPARTITIONED|
+                          -- SORT_MERGE_EXCHANGE [$$pid(ASC) ]  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$124]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- HASH_PARTITION_EXCHANGE [$$124]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$109]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- UNNEST  |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 (test.cart)  |PARTITIONED|
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- REPLICATE  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- DATASOURCE_SCAN (test.cart)  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_let_7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_let_7.plan
index 512a0e7..3febf26 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_let_7.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_let_7.plan
@@ -1,70 +1,60 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$111(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$111(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$118(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$118(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
               -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- SUBPLAN  |PARTITIONED|
-                            {
-                              -- AGGREGATE  |LOCAL|
-                                -- STREAM_SELECT  |LOCAL|
-                                  -- UNNEST  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                            }
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- UNNEST  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- NESTED_LOOP  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- NESTED_LOOP  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- UNNEST  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
                                 -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- UNNEST  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |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 (test.cart)  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- REPLICATE  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (test.cart)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- AGGREGATE  |UNPARTITIONED|
-                              -- SORT_MERGE_EXCHANGE [$$pid(ASC) ]  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- SORT_GROUP_BY[$$117]  |PARTITIONED|
-                                              {
-                                                -- AGGREGATE  |LOCAL|
-                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                              }
-                                        -- HASH_PARTITION_EXCHANGE [$$117]  |PARTITIONED|
-                                          -- SORT_GROUP_BY[$$102]  |PARTITIONED|
-                                                  {
-                                                    -- AGGREGATE  |LOCAL|
-                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                  }
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- STREAM_SELECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- UNNEST  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |UNPARTITIONED|
+                          -- SORT_MERGE_EXCHANGE [$$pid(ASC) ]  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$124]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- HASH_PARTITION_EXCHANGE [$$124]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$109]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- UNNEST  |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 (test.cart)  |PARTITIONED|
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- REPLICATE  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- DATASOURCE_SCAN (test.cart)  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- EMPTY_TUPLE_SOURCE  |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 641a491..4d5d792 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
@@ -1,72 +1,63 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$107(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$107(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$107(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$107(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
               -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |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|
-                      -- PRE_CLUSTERED_GROUP_BY[$$100, $$i1, $$103, $$99, $$110]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- STREAM_SELECT  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
+                      -- STABLE_SORT [$$100(ASC), $$i1(ASC), $$103(ASC), $$99(ASC), $$110(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$100(ASC), $$i1(ASC), $$103(ASC), $$99(ASC), $$110(ASC)]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$103][$$pid]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$103]  |PARTITIONED|
-                                      -- WINDOW_STREAM  |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|
+                              -- HYBRID_HASH_JOIN [$$103][$$pid]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$103]  |PARTITIONED|
+                                  -- WINDOW_STREAM  |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|
+                                            -- UNNEST  |PARTITIONED|
                                               -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- UNNEST  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- REPLICATE  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN (test.cart)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- SORT_GROUP_BY[$$118]  |PARTITIONED|
+                                                {
+                                                  -- AGGREGATE  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                }
+                                          -- HASH_PARTITION_EXCHANGE [$$118]  |PARTITIONED|
+                                            -- SORT_GROUP_BY[$$98]  |PARTITIONED|
+                                                    {
+                                                      -- AGGREGATE  |LOCAL|
+                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                    }
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_SELECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- UNNEST  |PARTITIONED|
                                                       -- ASSIGN  |PARTITIONED|
                                                         -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- REPLICATE  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (test.cart)  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- STREAM_SELECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- SORT_GROUP_BY[$$118]  |PARTITIONED|
-                                                      {
-                                                        -- AGGREGATE  |LOCAL|
-                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                      }
-                                                -- HASH_PARTITION_EXCHANGE [$$118]  |PARTITIONED|
-                                                  -- SORT_GROUP_BY[$$98]  |PARTITIONED|
-                                                          {
-                                                            -- AGGREGATE  |LOCAL|
-                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                          }
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- STREAM_SELECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- UNNEST  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ASSIGN  |PARTITIONED|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- REPLICATE  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- DATASOURCE_SCAN (test.cart)  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- REPLICATE  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN (test.cart)  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- 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 615f0c0..1f34548 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
@@ -1,78 +1,71 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$53(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$53(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$53(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$53(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |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[$$48]  |PARTITIONED|
-                                    {
-                                      -- AGGREGATE  |LOCAL|
-                                        -- STREAM_SELECT  |LOCAL|
-                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                    }
+                            -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- HYBRID_HASH_JOIN [$$48][$$37]  |PARTITIONED|
-                                          -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- STREAM_SELECT  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$48][$$37]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$48]  |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|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.Orders)  |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|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |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|
+                                    -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$48][$$37]  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
+                                                -- STREAM_SELECT  |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|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |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|
-                                        -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- HYBRID_HASH_JOIN [$$48][$$37]  |PARTITIONED|
-                                                  -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- STREAM_SELECT  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (test.Orders)  |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|
-                                                            -- DATASOURCE_SCAN (test.Orders)  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists.plan
index 5501c67..baadb28 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists.plan
@@ -1,74 +1,68 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$cntrycode(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$188]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$188]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$163]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- SORT_GROUP_BY[$$185]  |PARTITIONED|
-                                      {
-                                        -- AGGREGATE  |LOCAL|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$cntrycode(ASC) ]  |PARTITIONED|
+        -- SORT_GROUP_BY[$$197]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$197]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$172]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$194]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- HASH_PARTITION_EXCHANGE [$$194]  |PARTITIONED|
+                            -- PRE_CLUSTERED_GROUP_BY[$$188]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
                                           -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                      }
-                                -- HASH_PARTITION_EXCHANGE [$$185]  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$179]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$179(ASC)]  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$175][$$172]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$175]  |PARTITIONED|
-                                                  -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- NESTED_LOOP  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (test.Customer)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |UNPARTITIONED|
-                                                              -- ASSIGN  |UNPARTITIONED|
-                                                                -- AGGREGATE  |UNPARTITIONED|
-                                                                  -- AGGREGATE  |UNPARTITIONED|
-                                                                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                                      -- AGGREGATE  |PARTITIONED|
-                                                                        -- STREAM_SELECT  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- DATASOURCE_SCAN (test.Customer)  |PARTITIONED|
-                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$172]  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$188(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$184][$$181]  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$184]  |PARTITIONED|
+                                            -- RUNNING_AGGREGATE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- NESTED_LOOP  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (test.Orders)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (test.Customer)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                      -- ASSIGN  |UNPARTITIONED|
+                                                        -- AGGREGATE  |UNPARTITIONED|
+                                                          -- AGGREGATE  |UNPARTITIONED|
+                                                            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                              -- AGGREGATE  |PARTITIONED|
+                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- DATASOURCE_SCAN (test.Customer)  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$181]  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |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/not_exists_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists_ps.plan
index c8944e7..5a49132 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists_ps.plan
@@ -1,157 +1,146 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$cntrycode(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$cntrycode(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- SORT_GROUP_BY[$$188]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- HASH_PARTITION_EXCHANGE [$$188]  |PARTITIONED|
-                          -- SORT_GROUP_BY[$$163]  |PARTITIONED|
-                                  {
-                                    -- AGGREGATE  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                  }
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- STREAM_SELECT  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- SORT_GROUP_BY[$$185]  |PARTITIONED|
-                                                  {
-                                                    -- AGGREGATE  |LOCAL|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$cntrycode(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$cntrycode(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- SORT_GROUP_BY[$$197]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- HASH_PARTITION_EXCHANGE [$$197]  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$172]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$194]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- HASH_PARTITION_EXCHANGE [$$194]  |PARTITIONED|
+                                        -- PRE_CLUSTERED_GROUP_BY[$$188]  |PARTITIONED|
+                                                {
+                                                  -- AGGREGATE  |LOCAL|
+                                                    -- STREAM_SELECT  |LOCAL|
                                                       -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                  }
-                                            -- HASH_PARTITION_EXCHANGE [$$185]  |PARTITIONED|
-                                              -- PRE_CLUSTERED_GROUP_BY[$$179]  |PARTITIONED|
+                                                }
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$188(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$184][$$181]  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$184]  |PARTITIONED|
+                                                        -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- NESTED_LOOP  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- DATASOURCE_SCAN (test.Customer)  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |UNPARTITIONED|
+                                                                    -- AGGREGATE  |UNPARTITIONED|
+                                                                      -- AGGREGATE  |UNPARTITIONED|
+                                                                        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                                          -- AGGREGATE  |PARTITIONED|
+                                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- DATASOURCE_SCAN (test.Customer)  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$181]  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (test.Orders)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- SORT_GROUP_BY[$$197]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- HASH_PARTITION_EXCHANGE [$$197]  |PARTITIONED|
+                                  -- SORT_GROUP_BY[$$172]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- SORT_GROUP_BY[$$194]  |PARTITIONED|
                                                       {
                                                         -- AGGREGATE  |LOCAL|
-                                                          -- STREAM_SELECT  |LOCAL|
-                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                       }
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STABLE_SORT [$$179(ASC)]  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$194]  |PARTITIONED|
+                                                  -- PRE_CLUSTERED_GROUP_BY[$$188]  |PARTITIONED|
+                                                          {
+                                                            -- AGGREGATE  |LOCAL|
+                                                              -- STREAM_SELECT  |LOCAL|
+                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                          }
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- STABLE_SORT [$$188(ASC)]  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- HYBRID_HASH_JOIN [$$175][$$172]  |PARTITIONED|
-                                                            -- HASH_PARTITION_EXCHANGE [$$175]  |PARTITIONED|
-                                                              -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- NESTED_LOOP  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- HYBRID_HASH_JOIN [$$184][$$181]  |PARTITIONED|
+                                                                -- HASH_PARTITION_EXCHANGE [$$184]  |PARTITIONED|
+                                                                  -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                                    -- STREAM_PROJECT  |PARTITIONED|
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- ASSIGN  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- NESTED_LOOP  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                             -- ASSIGN  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- DATASOURCE_SCAN (test.Customer)  |PARTITIONED|
-                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |UNPARTITIONED|
-                                                                          -- ASSIGN  |UNPARTITIONED|
-                                                                            -- AGGREGATE  |UNPARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- DATASOURCE_SCAN (test.Customer)  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                            -- ASSIGN  |UNPARTITIONED|
                                                                               -- AGGREGATE  |UNPARTITIONED|
-                                                                                -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                                                  -- AGGREGATE  |PARTITIONED|
-                                                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                -- AGGREGATE  |UNPARTITIONED|
+                                                                                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                                                    -- AGGREGATE  |PARTITIONED|
+                                                                                      -- STREAM_SELECT  |PARTITIONED|
                                                                                         -- ASSIGN  |PARTITIONED|
                                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                             -- DATASOURCE_SCAN (test.Customer)  |PARTITIONED|
                                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                            -- HASH_PARTITION_EXCHANGE [$$172]  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (test.Orders)  |PARTITIONED|
+                                                                -- HASH_PARTITION_EXCHANGE [$$181]  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- SORT_GROUP_BY[$$188]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- HASH_PARTITION_EXCHANGE [$$188]  |PARTITIONED|
-                                    -- SORT_GROUP_BY[$$163]  |PARTITIONED|
-                                            {
-                                              -- AGGREGATE  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                            }
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- SORT_GROUP_BY[$$185]  |PARTITIONED|
-                                                            {
-                                                              -- AGGREGATE  |LOCAL|
-                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                            }
-                                                      -- HASH_PARTITION_EXCHANGE [$$185]  |PARTITIONED|
-                                                        -- PRE_CLUSTERED_GROUP_BY[$$179]  |PARTITIONED|
-                                                                {
-                                                                  -- AGGREGATE  |LOCAL|
-                                                                    -- STREAM_SELECT  |LOCAL|
-                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                }
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- STABLE_SORT [$$179(ASC)]  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- HYBRID_HASH_JOIN [$$175][$$172]  |PARTITIONED|
-                                                                      -- HASH_PARTITION_EXCHANGE [$$175]  |PARTITIONED|
-                                                                        -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- NESTED_LOOP  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                          -- DATASOURCE_SCAN (test.Customer)  |PARTITIONED|
-                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                  -- STREAM_PROJECT  |UNPARTITIONED|
-                                                                                    -- ASSIGN  |UNPARTITIONED|
-                                                                                      -- AGGREGATE  |UNPARTITIONED|
-                                                                                        -- AGGREGATE  |UNPARTITIONED|
-                                                                                          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                                                            -- AGGREGATE  |PARTITIONED|
-                                                                                              -- STREAM_SELECT  |PARTITIONED|
-                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                      -- DATASOURCE_SCAN (test.Customer)  |PARTITIONED|
-                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                      -- HASH_PARTITION_EXCHANGE [$$172]  |PARTITIONED|
-                                                                        -- ASSIGN  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- DATASOURCE_SCAN (test.Orders)  |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-1572-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-1572-2.plan
index 7dfcb75..06af916 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-1572-2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-1572-2.plan
@@ -1,12 +1,10 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN (sampdb.samptable)  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- DATASOURCE_SCAN (sampdb.samptable)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-1572.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-1572.plan
index bbafbac..27a4552 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-1572.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-1572.plan
@@ -1,29 +1,27 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- SUBPLAN  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- STREAM_SELECT  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
             -- SUBPLAN  |PARTITIONED|
                     {
                       -- AGGREGATE  |LOCAL|
                         -- STREAM_SELECT  |LOCAL|
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                     }
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- SUBPLAN  |PARTITIONED|
-                        {
-                          -- AGGREGATE  |LOCAL|
-                            -- STREAM_SELECT  |LOCAL|
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                        }
-                  -- STREAM_PROJECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (sampdb.samptable)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (sampdb.samptable)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |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 cb4bb4f..9b70fb7 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
@@ -1,74 +1,67 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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[$$83]  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$83(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- HYBRID_HASH_JOIN [$$83][$$94]  |PARTITIONED|
-                                -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |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[$$83]  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$83(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$83][$$94]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- REPLICATE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |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|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |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|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- HYBRID_HASH_JOIN [$$ve][$$86]  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$ve]  |PARTITIONED|
+                                                    -- UNNEST  |PARTITIONED|
                                                       -- STREAM_PROJECT  |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|
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- DATASOURCE_SCAN (test.Verification)  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                    -- HASH_PARTITION_EXCHANGE [$$86]  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- DATASOURCE_SCAN (test.Evidence)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- HYBRID_HASH_JOIN [$$92][$$88]  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$92]  |PARTITIONED|
+                                                              -- REPLICATE  |PARTITIONED|
                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- EMPTY_TUPLE_SOURCE  |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|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- DATASOURCE_SCAN (test.Verification)  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$86]  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |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 924d087..da70078 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
@@ -1,75 +1,68 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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[$$89]  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$89(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- HYBRID_HASH_JOIN [$$89][$$100]  |PARTITIONED|
-                                -- HASH_PARTITION_EXCHANGE [$$89]  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |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[$$89]  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$89(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$89][$$100]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$89]  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- REPLICATE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |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|
-                                                  -- 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|
+                                                  -- HYBRID_HASH_JOIN [$$ve][$$92]  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$ve]  |PARTITIONED|
+                                                      -- UNNEST  |PARTITIONED|
                                                         -- STREAM_PROJECT  |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|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- DATASOURCE_SCAN (test.Verification)  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                      -- HASH_PARTITION_EXCHANGE [$$92]  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- DATASOURCE_SCAN (test.Evidence)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- HYBRID_HASH_JOIN [$$98][$$94]  |PARTITIONED|
+                                                              -- HASH_PARTITION_EXCHANGE [$$98]  |PARTITIONED|
+                                                                -- REPLICATE  |PARTITIONED|
                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |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|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- DATASOURCE_SCAN (test.Verification)  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$92]  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |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 9aba5cf..74b9351 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
@@ -1,76 +1,68 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$80(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$80(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
               -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- PRE_CLUSTERED_GROUP_BY[$$75]  |PARTITIONED|
+                          {
+                            -- AGGREGATE  |LOCAL|
+                              -- STREAM_SELECT  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                          }
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$75]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- STREAM_SELECT  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
+                      -- STABLE_SORT [$$75(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$75(ASC)]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$75][$$87]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$75]  |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|
-                                                      -- DATASOURCE_SCAN (test.RawTweet)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$87]  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$75][$$87]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$75]  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- REPLICATE  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                          -- ASSIGN  |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|
-                                                            -- 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|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- DATASOURCE_SCAN (test.Verification)  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (test.Evidence)  |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|
+                                      -- 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|
+                                                      -- 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|
+                                                                    -- 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|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                      -- DATASOURCE_SCAN (test.Verification)  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |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 985399f..757c420 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
@@ -1,134 +1,122 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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 [$$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 [$$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 [$$296][$$jid]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$296]  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- REPLICATE  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (test.jds)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$jid]  |PARTITIONED|
+    -- ASSIGN  |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 [$$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 [$$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 [$$296][$$jid]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$296]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- REPLICATE  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- SORT_GROUP_BY[$$343, $$344]  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.jds)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$jid]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- SORT_GROUP_BY[$$343, $$344]  |PARTITIONED|
+                                                  {
+                                                    -- AGGREGATE  |LOCAL|
+                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                  }
+                                            -- HASH_PARTITION_EXCHANGE [$$343, $$344]  |PARTITIONED|
+                                              -- SORT_GROUP_BY[$$292, $$293]  |PARTITIONED|
                                                       {
                                                         -- AGGREGATE  |LOCAL|
                                                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                       }
-                                                -- HASH_PARTITION_EXCHANGE [$$343, $$344]  |PARTITIONED|
-                                                  -- SORT_GROUP_BY[$$292, $$293]  |PARTITIONED|
-                                                          {
-                                                            -- AGGREGATE  |LOCAL|
-                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                          }
-                                                    -- 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|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- DATASOURCE_SCAN (test.mds)  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- NESTED_LOOP  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_SELECT  |PARTITIONED|
-                            -- WINDOW_STREAM  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |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 [$$322][$$325]  |PARTITIONED|
-                                          -- HASH_PARTITION_EXCHANGE [$$322]  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- REPLICATE  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (test.jds)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                          -- HASH_PARTITION_EXCHANGE [$$325]  |PARTITIONED|
+                                                        -- REPLICATE  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN (test.mds)  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- NESTED_LOOP  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- WINDOW_STREAM  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |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 [$$322][$$325]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$322]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- REPLICATE  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (test.jds)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$325]  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
                                             -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- SORT_GROUP_BY[$$346, $$347]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- SORT_GROUP_BY[$$346, $$347]  |PARTITIONED|
+                                                        {
+                                                          -- AGGREGATE  |LOCAL|
+                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                        }
+                                                  -- HASH_PARTITION_EXCHANGE [$$346, $$347]  |PARTITIONED|
+                                                    -- SORT_GROUP_BY[$$330, $$331]  |PARTITIONED|
                                                             {
                                                               -- AGGREGATE  |LOCAL|
                                                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                             }
-                                                      -- HASH_PARTITION_EXCHANGE [$$346, $$347]  |PARTITIONED|
-                                                        -- SORT_GROUP_BY[$$330, $$331]  |PARTITIONED|
-                                                                {
-                                                                  -- AGGREGATE  |LOCAL|
-                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                }
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |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 (test.mds)  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- SORT_GROUP_BY[$$349, $$350]  |PARTITIONED|
-                                            {
-                                              -- AGGREGATE  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                            }
-                                      -- HASH_PARTITION_EXCHANGE [$$349, $$350]  |PARTITIONED|
-                                        -- SORT_GROUP_BY[$$294, $$295]  |PARTITIONED|
-                                                {
-                                                  -- AGGREGATE  |LOCAL|
-                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                }
+                                                            -- REPLICATE  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- DATASOURCE_SCAN (test.mds)  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- SORT_GROUP_BY[$$349, $$350]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- HASH_PARTITION_EXCHANGE [$$349, $$350]  |PARTITIONED|
+                                  -- SORT_GROUP_BY[$$294, $$295]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |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 (test.mds)  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (test.mds)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-3006.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-3006.plan
index dfaa310..0abba86 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-3006.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-3006.plan
@@ -1,41 +1,38 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$55(ASC) ]  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- SORT_GROUP_BY[$$65]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- HASH_PARTITION_EXCHANGE [$$65]  |PARTITIONED|
-                  -- PRE_CLUSTERED_GROUP_BY[$$46]  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- HYBRID_HASH_JOIN [$$b][$$48]  |PARTITIONED|
-                                -- HASH_PARTITION_EXCHANGE [$$b]  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- UNNEST  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$58(ASC) ]  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- SORT_GROUP_BY[$$68]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- HASH_PARTITION_EXCHANGE [$$68]  |PARTITIONED|
+                -- PRE_CLUSTERED_GROUP_BY[$$49]  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$49(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$b][$$51]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$b]  |PARTITIONED|
+                                -- UNNEST  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.ds1)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.ds1)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$51]  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.ds2)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.ds2)  |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/temporal/interval_joins/interval_after/interval_after.3.query.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_after/interval_after.3.query.plan
index 516f5ac..dcb2808 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_after/interval_after.3.query.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_after/interval_after.3.query.plan
@@ -1,31 +1,28 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- INTERVAL_MERGE_JOIN [$$32] [$$33]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- RANGE_PARTITION_EXCHANGE [$$36(ASC)] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- INTERVAL_MERGE_JOIN [$$34] [$$35]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- RANGE_PARTITION_EXCHANGE [$$38(ASC)] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
                         -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- PARTIAL_BROADCAST_RANGE_FOLLOWING_EXCHANGE [$$38(ASC)] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- PARTIAL_BROADCAST_RANGE_FOLLOWING_EXCHANGE [$$40(ASC)] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
                         -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_before/interval_before.3.query.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_before/interval_before.3.query.plan
index 04ec25e..b2b55dc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_before/interval_before.3.query.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_before/interval_before.3.query.plan
@@ -1,31 +1,28 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- INTERVAL_MERGE_JOIN [$$32] [$$33]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- PARTIAL_BROADCAST_RANGE_FOLLOWING_EXCHANGE [$$36(ASC)] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- INTERVAL_MERGE_JOIN [$$34] [$$35]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- PARTIAL_BROADCAST_RANGE_FOLLOWING_EXCHANGE [$$38(ASC)] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
                         -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- RANGE_PARTITION_EXCHANGE [$$38(ASC)] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- RANGE_PARTITION_EXCHANGE [$$40(ASC)] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
                         -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_covered_by/interval_covered_by.3.query.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_covered_by/interval_covered_by.3.query.plan
index 2c9e100..1be636a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_covered_by/interval_covered_by.3.query.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_covered_by/interval_covered_by.3.query.plan
@@ -1,31 +1,28 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- INTERVAL_MERGE_JOIN [$$32] [$$33]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- RANGE_PARTITION_EXCHANGE [$$36(ASC)] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- INTERVAL_MERGE_JOIN [$$34] [$$35]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- RANGE_PARTITION_EXCHANGE [$$38(ASC)] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
                         -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- PARTIAL_BROADCAST_RANGE_INTERSECT_EXCHANGE [{$$38,$$39,ASC}] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- PARTIAL_BROADCAST_RANGE_INTERSECT_EXCHANGE [{$$40,$$41,ASC}] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
                         -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_covers/interval_covers.3.query.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_covers/interval_covers.3.query.plan
index 90c6f12..5fed08f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_covers/interval_covers.3.query.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_covers/interval_covers.3.query.plan
@@ -1,31 +1,28 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- INTERVAL_MERGE_JOIN [$$32] [$$33]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- PARTIAL_BROADCAST_RANGE_INTERSECT_EXCHANGE [{$$36,$$37,ASC}] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- INTERVAL_MERGE_JOIN [$$34] [$$35]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- PARTIAL_BROADCAST_RANGE_INTERSECT_EXCHANGE [{$$38,$$39,ASC}] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
                         -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- RANGE_PARTITION_EXCHANGE [$$38(ASC)] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- RANGE_PARTITION_EXCHANGE [$$40(ASC)] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
                         -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_ended_by/interval_ended_by.3.query.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_ended_by/interval_ended_by.3.query.plan
index 2f858cf..96456f1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_ended_by/interval_ended_by.3.query.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_ended_by/interval_ended_by.3.query.plan
@@ -1,27 +1,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- HYBRID_HASH_JOIN [$$36][$$37]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$38][$$39]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_ends/interval_ends.3.query.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_ends/interval_ends.3.query.plan
index 2f858cf..96456f1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_ends/interval_ends.3.query.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_ends/interval_ends.3.query.plan
@@ -1,27 +1,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- HYBRID_HASH_JOIN [$$36][$$37]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$38][$$39]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_meets/interval_meets.3.query.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_meets/interval_meets.3.query.plan
index a5761ed..c9ecb47 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_meets/interval_meets.3.query.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_meets/interval_meets.3.query.plan
@@ -1,21 +1,18 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$32][$$33]  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$32]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- HYBRID_HASH_JOIN [$$34][$$35]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$33]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_met_by/interval_met_by.3.query.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_met_by/interval_met_by.3.query.plan
index a5761ed..c9ecb47 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_met_by/interval_met_by.3.query.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_met_by/interval_met_by.3.query.plan
@@ -1,21 +1,18 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$32][$$33]  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$32]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- HYBRID_HASH_JOIN [$$34][$$35]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$33]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.3.query.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.3.query.plan
index 2c9e100..1be636a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.3.query.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.3.query.plan
@@ -1,31 +1,28 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- INTERVAL_MERGE_JOIN [$$32] [$$33]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- RANGE_PARTITION_EXCHANGE [$$36(ASC)] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- INTERVAL_MERGE_JOIN [$$34] [$$35]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- RANGE_PARTITION_EXCHANGE [$$38(ASC)] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
                         -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- PARTIAL_BROADCAST_RANGE_INTERSECT_EXCHANGE [{$$38,$$39,ASC}] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- PARTIAL_BROADCAST_RANGE_INTERSECT_EXCHANGE [{$$40,$$41,ASC}] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
                         -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_overlapping/interval_overlapping.3.query.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_overlapping/interval_overlapping.3.query.plan
index cd143c5..6f42f1f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_overlapping/interval_overlapping.3.query.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_overlapping/interval_overlapping.3.query.plan
@@ -1,31 +1,28 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- INTERVAL_MERGE_JOIN [$$32] [$$33]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- PARTIAL_BROADCAST_RANGE_INTERSECT_EXCHANGE [{$$36,$$37,ASC}] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- INTERVAL_MERGE_JOIN [$$34] [$$35]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- PARTIAL_BROADCAST_RANGE_INTERSECT_EXCHANGE [{$$38,$$39,ASC}] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
                         -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- PARTIAL_BROADCAST_RANGE_INTERSECT_EXCHANGE [{$$38,$$39,ASC}] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- PARTIAL_BROADCAST_RANGE_INTERSECT_EXCHANGE [{$$40,$$41,ASC}] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
                         -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_overlaps/interval_overlaps.3.query.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_overlaps/interval_overlaps.3.query.plan
index 90c6f12..5fed08f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_overlaps/interval_overlaps.3.query.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_overlaps/interval_overlaps.3.query.plan
@@ -1,31 +1,28 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- INTERVAL_MERGE_JOIN [$$32] [$$33]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- PARTIAL_BROADCAST_RANGE_INTERSECT_EXCHANGE [{$$36,$$37,ASC}] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- INTERVAL_MERGE_JOIN [$$34] [$$35]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- PARTIAL_BROADCAST_RANGE_INTERSECT_EXCHANGE [{$$38,$$39,ASC}] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
                         -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- RANGE_PARTITION_EXCHANGE [$$38(ASC)] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- RANGE_PARTITION_EXCHANGE [$$40(ASC)] RANGE_MAP:{SPLIT:3}  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
                         -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_started_by/interval_started_by.3.query.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_started_by/interval_started_by.3.query.plan
index 2dce309..de9130d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_started_by/interval_started_by.3.query.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_started_by/interval_started_by.3.query.plan
@@ -1,27 +1,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- HYBRID_HASH_JOIN [$$34][$$35]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$36][$$37]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_starts/interval_starts.3.query.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_starts/interval_starts.3.query.plan
index 2dce309..de9130d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_starts/interval_starts.3.query.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_starts/interval_starts.3.query.plan
@@ -1,27 +1,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- HYBRID_HASH_JOIN [$$34][$$35]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$36][$$37]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (TinyCollege.Staff)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (TinyCollege.Students)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
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 cbeb55d..329988f 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
@@ -1,73 +1,69 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     -- STREAM_LIMIT  |UNPARTITIONED|
-      -- STREAM_PROJECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$$136(ASC) ]  |PARTITIONED|
-            -- STREAM_LIMIT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |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[$$145]  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$136(ASC) ]  |PARTITIONED|
+          -- STREAM_LIMIT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [topK: 100] [$$136(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$145]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- HASH_PARTITION_EXCHANGE [$$145]  |PARTITIONED|
+                          -- SORT_GROUP_BY[$$126]  |PARTITIONED|
                                   {
                                     -- AGGREGATE  |LOCAL|
                                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                                   }
-                            -- HASH_PARTITION_EXCHANGE [$$145]  |PARTITIONED|
-                              -- SORT_GROUP_BY[$$126]  |PARTITIONED|
-                                      {
-                                        -- AGGREGATE  |LOCAL|
-                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                      }
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- HYBRID_HASH_JOIN [$$129][$$132]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$129]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$137][$$131]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- HYBRID_HASH_JOIN [$$128][$$142]  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$128]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |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 [$$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 [$$142]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$129][$$132]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$129]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$137][$$131]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- HYBRID_HASH_JOIN [$$128][$$142]  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$128]  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |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.store_sales)  |PARTITIONED|
+                                                                  -- DATASOURCE_SCAN (tpcds.customer_address)  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                       -- EMPTY_TUPLE_SOURCE  |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|
-                                        -- HASH_PARTITION_EXCHANGE [$$132]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (tpcds.item)  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$140]  |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 [$$142]  |PARTITIONED|
+                                                      -- ASSIGN  |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 [$$131]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (tpcds.date_dim)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$132]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |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-1581-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581-2.plan
index 6e589bc..1daf3a5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581-2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581-2.plan
@@ -1,113 +1,38 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- UNNEST  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- NESTED_LOOP  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- NESTED_LOOP  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- NESTED_LOOP  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (tpcds.item.item)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                              -- AGGREGATE  |UNPARTITIONED|
-                                -- AGGREGATE  |UNPARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                    -- NESTED_LOOP  |UNPARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                        -- STREAM_PROJECT  |UNPARTITIONED|
-                                          -- STREAM_SELECT  |UNPARTITIONED|
-                                            -- STREAM_PROJECT  |UNPARTITIONED|
-                                              -- ASSIGN  |UNPARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                  -- REPLICATE  |UNPARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                      -- AGGREGATE  |UNPARTITIONED|
-                                                        -- AGGREGATE  |UNPARTITIONED|
-                                                          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                            -- AGGREGATE  |PARTITIONED|
-                                                              -- STREAM_SELECT  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ASSIGN  |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 (tpcds.store_sales)  |PARTITIONED|
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                        -- STREAM_PROJECT  |UNPARTITIONED|
-                                          -- ASSIGN  |UNPARTITIONED|
-                                            -- AGGREGATE  |UNPARTITIONED|
-                                              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                -- AGGREGATE  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- STREAM_SELECT  |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 (tpcds.store_sales)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- UNNEST  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- NESTED_LOOP  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- NESTED_LOOP  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (tpcds.item.item)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |UNPARTITIONED|
                           -- AGGREGATE  |UNPARTITIONED|
-                            -- AGGREGATE  |UNPARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                -- NESTED_LOOP  |UNPARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                    -- STREAM_PROJECT  |UNPARTITIONED|
-                                      -- STREAM_SELECT  |UNPARTITIONED|
-                                        -- STREAM_PROJECT  |UNPARTITIONED|
-                                          -- ASSIGN  |UNPARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                              -- REPLICATE  |UNPARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                  -- AGGREGATE  |UNPARTITIONED|
-                                                    -- AGGREGATE  |UNPARTITIONED|
-                                                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                        -- AGGREGATE  |PARTITIONED|
-                                                          -- STREAM_SELECT  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ASSIGN  |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 (tpcds.store_sales)  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                    -- STREAM_PROJECT  |UNPARTITIONED|
-                                      -- ASSIGN  |UNPARTITIONED|
-                                        -- AGGREGATE  |UNPARTITIONED|
-                                          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                            -- AGGREGATE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- STREAM_SELECT  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                              -- NESTED_LOOP  |UNPARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                  -- STREAM_SELECT  |UNPARTITIONED|
+                                    -- ASSIGN  |UNPARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                        -- REPLICATE  |UNPARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                            -- AGGREGATE  |UNPARTITIONED|
+                                              -- AGGREGATE  |UNPARTITIONED|
+                                                -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                  -- AGGREGATE  |PARTITIONED|
+                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
                                                         -- ASSIGN  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                             -- REPLICATE  |PARTITIONED|
@@ -117,26 +42,80 @@
                                                                     -- DATASOURCE_SCAN (tpcds.store_sales)  |PARTITIONED|
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |UNPARTITIONED|
-                        -- ASSIGN  |UNPARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                            -- REPLICATE  |UNPARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                -- AGGREGATE  |UNPARTITIONED|
-                                  -- AGGREGATE  |UNPARTITIONED|
-                                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                      -- AGGREGATE  |PARTITIONED|
-                                        -- STREAM_SELECT  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                  -- ASSIGN  |UNPARTITIONED|
+                                    -- AGGREGATE  |UNPARTITIONED|
+                                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                        -- AGGREGATE  |PARTITIONED|
+                                          -- STREAM_SELECT  |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|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- AGGREGATE  |UNPARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                          -- NESTED_LOOP  |UNPARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                              -- STREAM_SELECT  |UNPARTITIONED|
+                                -- ASSIGN  |UNPARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                    -- REPLICATE  |UNPARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                        -- AGGREGATE  |UNPARTITIONED|
+                                          -- AGGREGATE  |UNPARTITIONED|
+                                            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                              -- AGGREGATE  |PARTITIONED|
+                                                -- STREAM_SELECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- REPLICATE  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |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  |UNPARTITIONED|
+                              -- ASSIGN  |UNPARTITIONED|
+                                -- AGGREGATE  |UNPARTITIONED|
+                                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                    -- AGGREGATE  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- ASSIGN  |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 (tpcds.store_sales)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |UNPARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                    -- REPLICATE  |UNPARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                        -- AGGREGATE  |UNPARTITIONED|
+                          -- AGGREGATE  |UNPARTITIONED|
+                            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                              -- AGGREGATE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |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 (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-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581-correlated-2.plan
index b010fca..e0a721d 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
@@ -1,183 +1,179 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- UNNEST  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- PRE_CLUSTERED_GROUP_BY[$$234]  |PARTITIONED|
-                            {
-                              -- AGGREGATE  |LOCAL|
-                                -- AGGREGATE  |LOCAL|
-                                  -- STREAM_SELECT  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                            }
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STABLE_SORT [$$234(ASC)]  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- HYBRID_HASH_JOIN [$$234][$$182]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |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 [$$196][$$232]  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- PRE_CLUSTERED_GROUP_BY[$$144]  |PARTITIONED|
-                                                        {
-                                                          -- AGGREGATE  |LOCAL|
-                                                            -- AGGREGATE  |LOCAL|
-                                                              -- STREAM_SELECT  |LOCAL|
-                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                        }
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STABLE_SORT [$$144(ASC)]  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- HYBRID_HASH_JOIN [$$144][$$194]  |PARTITIONED|
-                                                              -- HASH_PARTITION_EXCHANGE [$$144]  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- UNNEST  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- PRE_CLUSTERED_GROUP_BY[$$234]  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$234(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$234][$$182]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |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 [$$196][$$232]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- PRE_CLUSTERED_GROUP_BY[$$144]  |PARTITIONED|
+                                                  {
+                                                    -- AGGREGATE  |LOCAL|
+                                                      -- AGGREGATE  |LOCAL|
+                                                        -- STREAM_SELECT  |LOCAL|
+                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                  }
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STABLE_SORT [$$144(ASC)]  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- HYBRID_HASH_JOIN [$$144][$$194]  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$144]  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |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|
-                                                                          -- 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|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$194]  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- SORT_GROUP_BY[$$240, $$241]  |PARTITIONED|
+                                                                        {
+                                                                          -- AGGREGATE  |LOCAL|
+                                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                        }
+                                                                  -- 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|
-                                                                        -- SORT_GROUP_BY[$$240, $$241]  |PARTITIONED|
-                                                                                {
-                                                                                  -- AGGREGATE  |LOCAL|
-                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                }
-                                                                          -- HASH_PARTITION_EXCHANGE [$$240, $$241]  |PARTITIONED|
-                                                                            -- PRE_CLUSTERED_GROUP_BY[$$188, $$193]  |PARTITIONED|
-                                                                                    {
-                                                                                      -- AGGREGATE  |LOCAL|
-                                                                                        -- STREAM_SELECT  |LOCAL|
-                                                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                    }
+                                                                        -- STABLE_SORT [$$188(ASC), $$193(ASC)]  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- STABLE_SORT [$$188(ASC), $$193(ASC)]  |PARTITIONED|
+                                                                                -- HYBRID_HASH_JOIN [$$177][$$176]  |PARTITIONED|
                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                    -- ASSIGN  |PARTITIONED|
                                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- HYBRID_HASH_JOIN [$$177][$$176]  |PARTITIONED|
+                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                          -- HASH_PARTITION_EXCHANGE [$$211]  |PARTITIONED|
+                                                                                            -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$215]  |PARTITIONED|
+                                                                                                              {
+                                                                                                                -- AGGREGATE  |LOCAL|
+                                                                                                                  -- AGGREGATE  |LOCAL|
+                                                                                                                    -- STREAM_SELECT  |LOCAL|
+                                                                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                              }
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- STABLE_SORT [$$215(ASC)]  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- STREAM_PROJECT  |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|
+                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$214]  |PARTITIONED|
+                                                                                                                              -- ASSIGN  |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 (tpcds.store_sales)  |PARTITIONED|
+                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                  -- HASH_PARTITION_EXCHANGE [$$176]  |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|
-                                                                                                    -- HASH_PARTITION_EXCHANGE [$$211]  |PARTITIONED|
-                                                                                                      -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                          -- ASSIGN  |PARTITIONED|
-                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                              -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$215]  |PARTITIONED|
-                                                                                                                              {
-                                                                                                                                -- AGGREGATE  |LOCAL|
-                                                                                                                                  -- AGGREGATE  |LOCAL|
-                                                                                                                                    -- STREAM_SELECT  |LOCAL|
-                                                                                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                                              }
-                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                          -- STABLE_SORT [$$215(ASC)]  |PARTITIONED|
-                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                              -- STREAM_PROJECT  |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|
-                                                                                                                                                  -- 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|
-                                                                                                                                                          -- REPLICATE  |PARTITIONED|
-                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |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|
-                                                                                                  -- REPLICATE  |PARTITIONED|
-                                                                                                    -- ONE_TO_ONE_EXCHANGE  |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 [$$232]  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |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 [$$232]  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- SORT_GROUP_BY[$$246, $$247]  |PARTITIONED|
+                                                        {
+                                                          -- AGGREGATE  |LOCAL|
+                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                        }
+                                                  -- 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|
-                                                        -- SORT_GROUP_BY[$$246, $$247]  |PARTITIONED|
-                                                                {
-                                                                  -- AGGREGATE  |LOCAL|
-                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                }
-                                                          -- HASH_PARTITION_EXCHANGE [$$246, $$247]  |PARTITIONED|
-                                                            -- PRE_CLUSTERED_GROUP_BY[$$198, $$231]  |PARTITIONED|
-                                                                    {
-                                                                      -- AGGREGATE  |LOCAL|
-                                                                        -- STREAM_SELECT  |LOCAL|
-                                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                    }
+                                                        -- STABLE_SORT [$$198(ASC), $$231(ASC)]  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- STABLE_SORT [$$198(ASC), $$231(ASC)]  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |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|
+                                                                -- HYBRID_HASH_JOIN [$$181][$$180]  |PARTITIONED|
+                                                                  -- HASH_PARTITION_EXCHANGE [$$181]  |PARTITIONED|
+                                                                    -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- PRE_CLUSTERED_GROUP_BY[$$201]  |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 [$$201][$$178]  |PARTITIONED|
                                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- PRE_CLUSTERED_GROUP_BY[$$201]  |PARTITIONED|
+                                                                                        -- PRE_CLUSTERED_GROUP_BY[$$202]  |PARTITIONED|
                                                                                                 {
                                                                                                   -- AGGREGATE  |LOCAL|
                                                                                                     -- AGGREGATE  |LOCAL|
@@ -185,169 +181,140 @@
                                                                                                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                                                                 }
                                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                            -- STABLE_SORT [$$202(ASC)]  |PARTITIONED|
                                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                -- HYBRID_HASH_JOIN [$$201][$$178]  |PARTITIONED|
+                                                                                                -- STREAM_PROJECT  |PARTITIONED|
                                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                    -- PRE_CLUSTERED_GROUP_BY[$$202]  |PARTITIONED|
-                                                                                                            {
-                                                                                                              -- AGGREGATE  |LOCAL|
-                                                                                                                -- AGGREGATE  |LOCAL|
-                                                                                                                  -- STREAM_SELECT  |LOCAL|
-                                                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                            }
+                                                                                                    -- HYBRID_HASH_JOIN [$$202][$$204]  |PARTITIONED|
                                                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                        -- STABLE_SORT [$$202(ASC)]  |PARTITIONED|
+                                                                                                        -- ASSIGN  |PARTITIONED|
                                                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                -- HYBRID_HASH_JOIN [$$202][$$204]  |PARTITIONED|
+                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                              -- HASH_PARTITION_EXCHANGE [$$215]  |PARTITIONED|
+                                                                                                                -- REPLICATE  |PARTITIONED|
                                                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                      -- ASSIGN  |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[$$243, $$244]  |PARTITIONED|
+                                                                                                                      {
+                                                                                                                        -- AGGREGATE  |LOCAL|
+                                                                                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                      }
+                                                                                                                -- 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 [$$209(ASC), $$210(ASC)]  |PARTITIONED|
                                                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                          -- REPLICATE  |PARTITIONED|
-                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$215]  |PARTITIONED|
-                                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                              -- HYBRID_HASH_JOIN [$$211][$$214]  |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 [$$204]  |PARTITIONED|
-                                                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                          -- SORT_GROUP_BY[$$243, $$244]  |PARTITIONED|
-                                                                                                                                  {
-                                                                                                                                    -- AGGREGATE  |LOCAL|
-                                                                                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                                                  }
-                                                                                                                            -- 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 [$$209(ASC), $$210(ASC)]  |PARTITIONED|
-                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                          -- HYBRID_HASH_JOIN [$$211][$$214]  |PARTITIONED|
-                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                              -- REPLICATE  |PARTITIONED|
-                                                                                                                                                -- HASH_PARTITION_EXCHANGE [$$211]  |PARTITIONED|
-                                                                                                                                                  -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$211]  |PARTITIONED|
+                                                                                                                                      -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                -- PRE_CLUSTERED_GROUP_BY[$$215]  |PARTITIONED|
+                                                                                                                                                        {
+                                                                                                                                                          -- AGGREGATE  |LOCAL|
+                                                                                                                                                            -- AGGREGATE  |LOCAL|
+                                                                                                                                                              -- STREAM_SELECT  |LOCAL|
+                                                                                                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                                        }
+                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                    -- STABLE_SORT [$$215(ASC)]  |PARTITIONED|
+                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                                         -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                          -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                  -- PRE_CLUSTERED_GROUP_BY[$$215]  |PARTITIONED|
-                                                                                                                                                                          {
-                                                                                                                                                                            -- AGGREGATE  |LOCAL|
-                                                                                                                                                                              -- AGGREGATE  |LOCAL|
-                                                                                                                                                                                -- STREAM_SELECT  |LOCAL|
-                                                                                                                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                                                                                          }
-                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                      -- STABLE_SORT [$$215(ASC)]  |PARTITIONED|
-                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                          -- STREAM_PROJECT  |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|
+                                                                                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$214]  |PARTITIONED|
+                                                                                                                                                                        -- ASSIGN  |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|
-                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                              -- REPLICATE  |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|
-                                                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |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 [$$214]  |PARTITIONED|
-                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                    -- ASSIGN  |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 (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|
+                                                                                                                                                                                      -- 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 [$$214]  |PARTITIONED|
+                                                                                                                                      -- ASSIGN  |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 (tpcds.store_sales)  |PARTITIONED|
+                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                              -- HASH_PARTITION_EXCHANGE [$$214]  |PARTITIONED|
+                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                      -- REPLICATE  |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|
-                                                                                                                        -- REPLICATE  |PARTITIONED|
-                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |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 [$$180]  |PARTITIONED|
-                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                  -- ASSIGN  |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 [$$180]  |PARTITIONED|
+                                                                    -- ASSIGN  |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 (tpcds.store_sales)  |PARTITIONED|
                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- REPLICATE  |PARTITIONED|
-                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                              -- DATASOURCE_SCAN (tpcds.store_sales)  |PARTITIONED|
-                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- REPLICATE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$214]  |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|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- REPLICATE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (tpcds.store_sales)  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- REPLICATE  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- DATASOURCE_SCAN (tpcds.store_sales)  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |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 de951d4..1a8fb37 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
@@ -1,351 +1,319 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- UNNEST  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- PRE_CLUSTERED_GROUP_BY[$$180]  |PARTITIONED|
-                            {
-                              -- AGGREGATE  |LOCAL|
-                                -- AGGREGATE  |LOCAL|
-                                  -- STREAM_SELECT  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                            }
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STABLE_SORT [$$180(ASC)]  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- HYBRID_HASH_JOIN [$$180][$$222]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |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|
-                                            -- HYBRID_HASH_JOIN [$$164][$$178]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- UNNEST  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- PRE_CLUSTERED_GROUP_BY[$$180]  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$180(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$180][$$222]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |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|
+                                      -- HYBRID_HASH_JOIN [$$164][$$178]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- PRE_CLUSTERED_GROUP_BY[$$144]  |PARTITIONED|
+                                                  {
+                                                    -- AGGREGATE  |LOCAL|
+                                                      -- AGGREGATE  |LOCAL|
+                                                        -- STREAM_SELECT  |LOCAL|
+                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                  }
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STABLE_SORT [$$144(ASC)]  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- HYBRID_HASH_JOIN [$$144][$$154]  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$144]  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |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|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- REPLICATE  |PARTITIONED|
+                                                                -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
+                                                                  -- ASSIGN  |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 (tpcds.store_sales)  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$178]  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- PRE_CLUSTERED_GROUP_BY[$$144]  |PARTITIONED|
+                                                -- SORT_GROUP_BY[$$227, $$228]  |PARTITIONED|
                                                         {
                                                           -- AGGREGATE  |LOCAL|
-                                                            -- AGGREGATE  |LOCAL|
-                                                              -- STREAM_SELECT  |LOCAL|
-                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                         }
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STABLE_SORT [$$144(ASC)]  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$227, $$228]  |PARTITIONED|
+                                                    -- PRE_CLUSTERED_GROUP_BY[$$165, $$168]  |PARTITIONED|
+                                                            {
+                                                              -- AGGREGATE  |LOCAL|
+                                                                -- STREAM_SELECT  |LOCAL|
+                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                            }
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- STABLE_SORT [$$165(ASC), $$168(ASC)]  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |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|
-                                                                          -- REPLICATE  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- BTREE_SEARCH (tpcds.item.item)  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |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|
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                -- ASSIGN  |PARTITIONED|
+                                                                -- HYBRID_HASH_JOIN [$$157][$$156]  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- REPLICATE  |PARTITIONED|
+                                                                          -- HASH_PARTITION_EXCHANGE [$$204]  |PARTITIONED|
+                                                                            -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- STREAM_SELECT  |PARTITIONED|
                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                     -- REPLICATE  |PARTITIONED|
                                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                        -- PRE_CLUSTERED_GROUP_BY[$$206]  |PARTITIONED|
+                                                                                                {
+                                                                                                  -- AGGREGATE  |LOCAL|
+                                                                                                    -- AGGREGATE  |LOCAL|
+                                                                                                      -- STREAM_SELECT  |LOCAL|
+                                                                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                }
                                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            -- DATASOURCE_SCAN (tpcds.store_sales)  |PARTITIONED|
+                                                                                            -- STABLE_SORT [$$206(ASC)]  |PARTITIONED|
                                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$178]  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- SORT_GROUP_BY[$$227, $$228]  |PARTITIONED|
-                                                                {
-                                                                  -- AGGREGATE  |LOCAL|
-                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                }
-                                                          -- HASH_PARTITION_EXCHANGE [$$227, $$228]  |PARTITIONED|
-                                                            -- PRE_CLUSTERED_GROUP_BY[$$165, $$168]  |PARTITIONED|
-                                                                    {
-                                                                      -- AGGREGATE  |LOCAL|
-                                                                        -- STREAM_SELECT  |LOCAL|
-                                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                    }
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- STABLE_SORT [$$165(ASC), $$168(ASC)]  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- HYBRID_HASH_JOIN [$$206][$$207]  |PARTITIONED|
+                                                                                                      -- HASH_PARTITION_EXCHANGE [$$206]  |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|
+                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                              -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
+                                                                                                                -- ASSIGN  |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 (tpcds.store_sales)  |PARTITIONED|
+                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                  -- HASH_PARTITION_EXCHANGE [$$156]  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- HYBRID_HASH_JOIN [$$157][$$156]  |PARTITIONED|
+                                                                        -- REPLICATE  |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|
-                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                              -- STREAM_SELECT  |PARTITIONED|
-                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                  -- REPLICATE  |PARTITIONED|
-                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$206]  |PARTITIONED|
-                                                                                                              {
-                                                                                                                -- AGGREGATE  |LOCAL|
-                                                                                                                  -- AGGREGATE  |LOCAL|
-                                                                                                                    -- STREAM_SELECT  |LOCAL|
-                                                                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                              }
-                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          -- STABLE_SORT [$$206(ASC)]  |PARTITIONED|
-                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                  -- HYBRID_HASH_JOIN [$$206][$$207]  |PARTITIONED|
-                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$206]  |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|
-                                                                                                                                    -- 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 (tpcds.store_sales)  |PARTITIONED|
-                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                          -- HASH_PARTITION_EXCHANGE [$$156]  |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 [$$222]  |PARTITIONED|
+                              -- ASSIGN  |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|
+                                                {
+                                                  -- AGGREGATE  |LOCAL|
+                                                    -- STREAM_SELECT  |LOCAL|
+                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                }
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$181(ASC), $$184(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$159][$$158]  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$159]  |PARTITIONED|
+                                                        -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- STREAM_SELECT  |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|
+                                                                          }
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- HYBRID_HASH_JOIN [$$185][$$188]  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                               -- ASSIGN  |PARTITIONED|
                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                   -- REPLICATE  |PARTITIONED|
                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$206]  |PARTITIONED|
+                                                                                              {
+                                                                                                -- AGGREGATE  |LOCAL|
+                                                                                                  -- AGGREGATE  |LOCAL|
+                                                                                                    -- STREAM_SELECT  |LOCAL|
+                                                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                              }
                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                          -- DATASOURCE_SCAN (tpcds.store_sales)  |PARTITIONED|
+                                                                                          -- STABLE_SORT [$$206(ASC)]  |PARTITIONED|
                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                  -- HASH_PARTITION_EXCHANGE [$$222]  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |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|
-                                                        {
-                                                          -- AGGREGATE  |LOCAL|
-                                                            -- STREAM_SELECT  |LOCAL|
-                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                        }
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STABLE_SORT [$$181(ASC), $$184(ASC)]  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |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|
-                                                                    -- ASSIGN  |PARTITIONED|
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        -- STREAM_SELECT  |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|
-                                                                                        }
-                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- HYBRID_HASH_JOIN [$$185][$$188]  |PARTITIONED|
-                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                  -- REPLICATE  |PARTITIONED|
-                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$206]  |PARTITIONED|
-                                                                                                              {
-                                                                                                                -- AGGREGATE  |LOCAL|
-                                                                                                                  -- AGGREGATE  |LOCAL|
-                                                                                                                    -- STREAM_SELECT  |LOCAL|
-                                                                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                              }
-                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          -- STABLE_SORT [$$206(ASC)]  |PARTITIONED|
-                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                  -- HYBRID_HASH_JOIN [$$206][$$207]  |PARTITIONED|
-                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$206]  |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|
-                                                                                                                                    -- 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 (tpcds.store_sales)  |PARTITIONED|
-                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                          -- HASH_PARTITION_EXCHANGE [$$188]  |PARTITIONED|
-                                                                                            -- ASSIGN  |PARTITIONED|
                                                                                               -- STREAM_PROJECT  |PARTITIONED|
                                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                  -- SORT_GROUP_BY[$$230, $$231]  |PARTITIONED|
-                                                                                                          {
-                                                                                                            -- AGGREGATE  |LOCAL|
-                                                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                          }
-                                                                                                    -- HASH_PARTITION_EXCHANGE [$$230, $$231]  |PARTITIONED|
-                                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$200, $$201]  |PARTITIONED|
-                                                                                                              {
-                                                                                                                -- AGGREGATE  |LOCAL|
-                                                                                                                  -- STREAM_SELECT  |LOCAL|
-                                                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                              }
+                                                                                                  -- HYBRID_HASH_JOIN [$$206][$$207]  |PARTITIONED|
+                                                                                                    -- HASH_PARTITION_EXCHANGE [$$206]  |PARTITIONED|
+                                                                                                      -- REPLICATE  |PARTITIONED|
                                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          -- STABLE_SORT [$$200(ASC), $$201(ASC)]  |PARTITIONED|
+                                                                                                          -- BTREE_SEARCH (tpcds.item.item)  |PARTITIONED|
                                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                  -- HYBRID_HASH_JOIN [$$204][$$205]  |PARTITIONED|
-                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                      -- REPLICATE  |PARTITIONED|
-                                                                                                                        -- HASH_PARTITION_EXCHANGE [$$204]  |PARTITIONED|
-                                                                                                                          -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                            -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
+                                                                                                              -- ASSIGN  |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 (tpcds.store_sales)  |PARTITIONED|
+                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                            -- HASH_PARTITION_EXCHANGE [$$188]  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- SORT_GROUP_BY[$$230, $$231]  |PARTITIONED|
+                                                                                            {
+                                                                                              -- AGGREGATE  |LOCAL|
+                                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                            }
+                                                                                      -- HASH_PARTITION_EXCHANGE [$$230, $$231]  |PARTITIONED|
+                                                                                        -- PRE_CLUSTERED_GROUP_BY[$$200, $$201]  |PARTITIONED|
+                                                                                                {
+                                                                                                  -- AGGREGATE  |LOCAL|
+                                                                                                    -- STREAM_SELECT  |LOCAL|
+                                                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                }
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- STABLE_SORT [$$200(ASC), $$201(ASC)]  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- HYBRID_HASH_JOIN [$$204][$$205]  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                          -- HASH_PARTITION_EXCHANGE [$$204]  |PARTITIONED|
+                                                                                                            -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                        -- PRE_CLUSTERED_GROUP_BY[$$206]  |PARTITIONED|
+                                                                                                                                {
+                                                                                                                                  -- AGGREGATE  |LOCAL|
+                                                                                                                                    -- AGGREGATE  |LOCAL|
+                                                                                                                                      -- STREAM_SELECT  |LOCAL|
+                                                                                                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                }
+                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                            -- STABLE_SORT [$$206(ASC)]  |PARTITIONED|
+                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                 -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                  -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                      -- REPLICATE  |PARTITIONED|
-                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                          -- PRE_CLUSTERED_GROUP_BY[$$206]  |PARTITIONED|
-                                                                                                                                                  {
-                                                                                                                                                    -- AGGREGATE  |LOCAL|
-                                                                                                                                                      -- AGGREGATE  |LOCAL|
-                                                                                                                                                        -- STREAM_SELECT  |LOCAL|
-                                                                                                                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                                                                  }
-                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                              -- STABLE_SORT [$$206(ASC)]  |PARTITIONED|
-                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                    -- HYBRID_HASH_JOIN [$$206][$$207]  |PARTITIONED|
+                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$206]  |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|
+                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
+                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                  -- ASSIGN  |PARTITIONED|
                                                                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                      -- HYBRID_HASH_JOIN [$$206][$$207]  |PARTITIONED|
-                                                                                                                                                        -- HASH_PARTITION_EXCHANGE [$$206]  |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|
+                                                                                                                                                      -- REPLICATE  |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|
-                                                                                                                                                                        -- 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 (tpcds.store_sales)  |PARTITIONED|
-                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |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|
+                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- REPLICATE  |PARTITIONED|
                                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                      -- REPLICATE  |PARTITIONED|
-                                                                                                                        -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
-                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                            -- ASSIGN  |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 (tpcds.store_sales)  |PARTITIONED|
-                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                              -- HASH_PARTITION_EXCHANGE [$$158]  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ASSIGN  |PARTITIONED|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                      -- ASSIGN  |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 [$$158]  |PARTITIONED|
+                                                        -- ASSIGN  |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 (tpcds.store_sales)  |PARTITIONED|
                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- REPLICATE  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- DATASOURCE_SCAN (tpcds.store_sales)  |PARTITIONED|
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581.plan
index 728c15a..6b377c1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581.plan
@@ -9,46 +9,146 @@
                 -- ASSIGN  |PARTITIONED|
                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
       -- BROADCAST_EXCHANGE  |LOCAL|
-        -- STREAM_PROJECT  |LOCAL|
-          -- ASSIGN  |LOCAL|
-            -- STREAM_PROJECT  |LOCAL|
-              -- UNNEST  |LOCAL|
-                -- STREAM_PROJECT  |LOCAL|
-                  -- ASSIGN  |LOCAL|
-                    -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                      -- PRE_CLUSTERED_GROUP_BY[$$166]  |LOCAL|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- AGGREGATE  |LOCAL|
-                                    -- STREAM_SELECT  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                          -- STREAM_PROJECT  |LOCAL|
-                            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                              -- HYBRID_HASH_JOIN [$$166][$$167]  |LOCAL|
-                                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$102]  |LOCAL|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- AGGREGATE  |LOCAL|
-                                                -- STREAM_SELECT  |LOCAL|
-                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                                      -- STABLE_SORT [$$102(ASC)]  |LOCAL|
-                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                          -- STREAM_PROJECT  |UNPARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$102][$$159]  |UNPARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                  -- STREAM_PROJECT  |UNPARTITIONED|
+        -- ASSIGN  |LOCAL|
+          -- UNNEST  |LOCAL|
+            -- ASSIGN  |LOCAL|
+              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                -- PRE_CLUSTERED_GROUP_BY[$$173]  |LOCAL|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- AGGREGATE  |LOCAL|
+                              -- STREAM_SELECT  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STREAM_PROJECT  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                        -- HYBRID_HASH_JOIN [$$173][$$174]  |LOCAL|
+                          -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                            -- PRE_CLUSTERED_GROUP_BY[$$109]  |LOCAL|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                                -- STABLE_SORT [$$109(ASC)]  |LOCAL|
+                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                    -- STREAM_PROJECT  |UNPARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$109][$$166]  |UNPARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                            -- ASSIGN  |UNPARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                -- REPLICATE  |UNPARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
                                                     -- ASSIGN  |UNPARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
                                                         -- REPLICATE  |UNPARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                            -- STREAM_PROJECT  |UNPARTITIONED|
-                                                              -- ASSIGN  |UNPARTITIONED|
+                                                            -- AGGREGATE  |UNPARTITIONED|
+                                                              -- AGGREGATE  |UNPARTITIONED|
+                                                                -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                                  -- AGGREGATE  |PARTITIONED|
+                                                                    -- STREAM_SELECT  |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 (tpcds.store_sales)  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                            -- NESTED_LOOP  |UNPARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                -- ASSIGN  |UNPARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                    -- REPLICATE  |UNPARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                        -- ASSIGN  |UNPARTITIONED|
+                                                          -- STREAM_SELECT  |UNPARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                              -- REPLICATE  |UNPARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                                  -- AGGREGATE  |UNPARTITIONED|
+                                                                    -- AGGREGATE  |UNPARTITIONED|
+                                                                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                                        -- AGGREGATE  |PARTITIONED|
+                                                                          -- STREAM_SELECT  |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 (tpcds.store_sales)  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                -- ASSIGN  |UNPARTITIONED|
+                                                  -- AGGREGATE  |UNPARTITIONED|
+                                                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                      -- AGGREGATE  |PARTITIONED|
+                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                          -- ASSIGN  |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 (tpcds.store_sales)  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                            -- NESTED_LOOP  |LOCAL|
+                              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                                -- ASSIGN  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- ASSIGN  |LOCAL|
+                                      -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                                        -- PRE_CLUSTERED_GROUP_BY[$$175]  |LOCAL|
+                                                {
+                                                  -- AGGREGATE  |LOCAL|
+                                                    -- AGGREGATE  |LOCAL|
+                                                      -- STREAM_SELECT  |LOCAL|
+                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                }
+                                          -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                                            -- STABLE_SORT [$$175(ASC)]  |LOCAL|
+                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                -- STREAM_PROJECT  |UNPARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$175][$$176]  |UNPARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                        -- REPLICATE  |UNPARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                            -- ASSIGN  |UNPARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                                -- REPLICATE  |UNPARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                                    -- AGGREGATE  |UNPARTITIONED|
+                                                                      -- AGGREGATE  |UNPARTITIONED|
+                                                                        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                                          -- AGGREGATE  |PARTITIONED|
+                                                                            -- STREAM_SELECT  |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 (tpcds.store_sales)  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                        -- REPLICATE  |UNPARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                            -- ASSIGN  |UNPARTITIONED|
+                                                              -- STREAM_SELECT  |UNPARTITIONED|
                                                                 -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
                                                                   -- REPLICATE  |UNPARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
@@ -57,148 +157,28 @@
                                                                           -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
                                                                             -- AGGREGATE  |PARTITIONED|
                                                                               -- STREAM_SELECT  |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 (tpcds.store_sales)  |PARTITIONED|
-                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                  -- NESTED_LOOP  |UNPARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                      -- STREAM_PROJECT  |UNPARTITIONED|
-                                                        -- ASSIGN  |UNPARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                            -- REPLICATE  |UNPARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                                -- ASSIGN  |UNPARTITIONED|
-                                                                  -- STREAM_SELECT  |UNPARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                                      -- REPLICATE  |UNPARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                                          -- AGGREGATE  |UNPARTITIONED|
-                                                                            -- AGGREGATE  |UNPARTITIONED|
-                                                                              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                                                -- AGGREGATE  |PARTITIONED|
-                                                                                  -- STREAM_SELECT  |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 (tpcds.store_sales)  |PARTITIONED|
-                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                      -- STREAM_PROJECT  |UNPARTITIONED|
-                                                        -- ASSIGN  |UNPARTITIONED|
-                                                          -- AGGREGATE  |UNPARTITIONED|
-                                                            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                              -- AGGREGATE  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- STREAM_SELECT  |PARTITIONED|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                      -- ASSIGN  |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 (tpcds.store_sales)  |PARTITIONED|
-                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                                  -- NESTED_LOOP  |LOCAL|
-                                    -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                                      -- ASSIGN  |LOCAL|
-                                        -- STREAM_PROJECT  |LOCAL|
-                                          -- STREAM_SELECT  |LOCAL|
-                                            -- ASSIGN  |LOCAL|
-                                              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                                                -- PRE_CLUSTERED_GROUP_BY[$$168]  |LOCAL|
-                                                        {
-                                                          -- AGGREGATE  |LOCAL|
-                                                            -- AGGREGATE  |LOCAL|
-                                                              -- STREAM_SELECT  |LOCAL|
-                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                        }
-                                                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                                                    -- STABLE_SORT [$$168(ASC)]  |LOCAL|
-                                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                        -- STREAM_PROJECT  |UNPARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                            -- HYBRID_HASH_JOIN [$$168][$$169]  |UNPARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                                -- REPLICATE  |UNPARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                                    -- STREAM_PROJECT  |UNPARTITIONED|
-                                                                      -- ASSIGN  |UNPARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                                          -- REPLICATE  |UNPARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                                              -- AGGREGATE  |UNPARTITIONED|
-                                                                                -- AGGREGATE  |UNPARTITIONED|
-                                                                                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                                                    -- AGGREGATE  |PARTITIONED|
-                                                                                      -- STREAM_SELECT  |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 (tpcds.store_sales)  |PARTITIONED|
-                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                                -- REPLICATE  |UNPARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                                    -- ASSIGN  |UNPARTITIONED|
-                                                                      -- STREAM_SELECT  |UNPARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                                          -- REPLICATE  |UNPARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                                              -- AGGREGATE  |UNPARTITIONED|
-                                                                                -- AGGREGATE  |UNPARTITIONED|
-                                                                                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                                                    -- AGGREGATE  |PARTITIONED|
-                                                                                      -- STREAM_SELECT  |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 (tpcds.store_sales)  |PARTITIONED|
-                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                      -- STREAM_PROJECT  |UNPARTITIONED|
-                                        -- ASSIGN  |UNPARTITIONED|
-                                          -- AGGREGATE  |UNPARTITIONED|
-                                            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                              -- AGGREGATE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- DATASOURCE_SCAN (tpcds.store_sales)  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                -- ASSIGN  |UNPARTITIONED|
+                                  -- AGGREGATE  |UNPARTITIONED|
+                                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                      -- AGGREGATE  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ASSIGN  |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|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- DATASOURCE_SCAN (tpcds.store_sales)  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |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 bdf16c1..0d83d24 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
@@ -1,151 +1,146 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     -- STREAM_LIMIT  |UNPARTITIONED|
-      -- STREAM_PROJECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- SORT_MERGE_EXCHANGE [$$192(ASC) ]  |PARTITIONED|
-              -- STREAM_LIMIT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$192(ASC) ]  |PARTITIONED|
+            -- STREAM_LIMIT  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- SORT_GROUP_BY[$$201, $$202]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- SORT_GROUP_BY[$$201, $$202]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- HASH_PARTITION_EXCHANGE [$$201, $$202]  |PARTITIONED|
+                        -- PRE_CLUSTERED_GROUP_BY[$$189, $$190]  |PARTITIONED|
                                 {
                                   -- AGGREGATE  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
                                 }
-                          -- 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 [$$189(ASC), $$190(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$189(ASC), $$190(ASC)]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- HYBRID_HASH_JOIN [$$189][$$171]  |PARTITIONED|
-                                          -- HASH_PARTITION_EXCHANGE [$$189]  |PARTITIONED|
-                                            -- SORT_GROUP_BY[$$198, $$199]  |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 [$$198, $$199]  |PARTITIONED|
+                                            -- PRE_CLUSTERED_GROUP_BY[$$186, $$187]  |PARTITIONED|
                                                     {
                                                       -- AGGREGATE  |LOCAL|
-                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                        -- STREAM_SELECT  |LOCAL|
+                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                     }
-                                              -- 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 [$$186(ASC), $$187(ASC)]  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STABLE_SORT [$$186(ASC), $$187(ASC)]  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |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[$$195, $$196]  |PARTITIONED|
-                                                                              {
-                                                                                -- AGGREGATE  |LOCAL|
+                                                        -- HYBRID_HASH_JOIN [$$186][$$169]  |PARTITIONED|
+                                                          -- HASH_PARTITION_EXCHANGE [$$186]  |PARTITIONED|
+                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- SORT_GROUP_BY[$$195, $$196]  |PARTITIONED|
+                                                                        {
+                                                                          -- AGGREGATE  |LOCAL|
+                                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                        }
+                                                                  -- HASH_PARTITION_EXCHANGE [$$195, $$196]  |PARTITIONED|
+                                                                    -- PRE_CLUSTERED_GROUP_BY[$$150, $$151]  |PARTITIONED|
+                                                                            {
+                                                                              -- AGGREGATE  |LOCAL|
+                                                                                -- STREAM_SELECT  |LOCAL|
                                                                                   -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                              }
-                                                                        -- 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 [$$150(ASC), $$151(ASC)]  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- HYBRID_HASH_JOIN [$$150][$$167]  |PARTITIONED|
-                                                                                        -- HASH_PARTITION_EXCHANGE [$$150]  |PARTITIONED|
-                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                            -- ONE_TO_ONE_EXCHANGE  |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|
-                                                                                                -- HASH_PARTITION_EXCHANGE [$$151]  |PARTITIONED|
-                                                                                                  -- DATASOURCE_SCAN (tpcds.customer_address)  |PARTITIONED|
-                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                        -- HASH_PARTITION_EXCHANGE [$$167]  |PARTITIONED|
-                                                                                          -- ASSIGN  |PARTITIONED|
-                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                -- HYBRID_HASH_JOIN [$$172][$$154]  |PARTITIONED|
-                                                                                                  -- HASH_PARTITION_EXCHANGE [$$172]  |PARTITIONED|
-                                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                        -- ONE_TO_ONE_EXCHANGE  |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 [$$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 [$$169]  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- HYBRID_HASH_JOIN [$$174][$$157]  |PARTITIONED|
-                                                                        -- HASH_PARTITION_EXCHANGE [$$174]  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            }
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- STABLE_SORT [$$150(ASC), $$151(ASC)]  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                             -- STREAM_PROJECT  |PARTITIONED|
                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- DATASOURCE_SCAN (tpcds.web_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 [$$154]  |PARTITIONED|
-                                                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                                                -- HYBRID_HASH_JOIN [$$150][$$167]  |PARTITIONED|
+                                                                                  -- HASH_PARTITION_EXCHANGE [$$150]  |PARTITIONED|
+                                                                                    -- STREAM_PROJECT  |PARTITIONED|
                                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- DATASOURCE_SCAN (tpcds.date_dim)  |PARTITIONED|
-                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                          -- HASH_PARTITION_EXCHANGE [$$171]  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_PROJECT  |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|
+                                                                                          -- HASH_PARTITION_EXCHANGE [$$151]  |PARTITIONED|
+                                                                                            -- DATASOURCE_SCAN (tpcds.customer_address)  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                  -- HASH_PARTITION_EXCHANGE [$$167]  |PARTITIONED|
+                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- HYBRID_HASH_JOIN [$$172][$$154]  |PARTITIONED|
+                                                                                            -- HASH_PARTITION_EXCHANGE [$$172]  |PARTITIONED|
+                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |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 [$$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 [$$169]  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- HYBRID_HASH_JOIN [$$174][$$157]  |PARTITIONED|
+                                                                    -- HASH_PARTITION_EXCHANGE [$$174]  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- DATASOURCE_SCAN (tpcds.web_sales)  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- REPLICATE  |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 [$$171]  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- HYBRID_HASH_JOIN [$$176][$$160]  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$176]  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN (tpcds.catalog_sales)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- HYBRID_HASH_JOIN [$$176][$$160]  |PARTITIONED|
-                                                    -- HASH_PARTITION_EXCHANGE [$$176]  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- DATASOURCE_SCAN (tpcds.catalog_sales)  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- REPLICATE  |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|
+                                                      -- REPLICATE  |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|
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 6b55682..286620f 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
@@ -1,27 +1,26 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$53][$$57]  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
-                        -- STREAM_PROJECT  |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|
-                              -- HASH_PARTITION_EXCHANGE [$$56, $$55]  |PARTITIONED|
-                                -- DATASOURCE_SCAN (tpcds.catalog_returns)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$57]  |PARTITIONED|
-                        -- DATASOURCE_SCAN (tpcds.item)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$53][$$57]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                      -- STREAM_PROJECT  |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|
+                            -- HASH_PARTITION_EXCHANGE [$$56, $$55]  |PARTITIONED|
+                              -- DATASOURCE_SCAN (tpcds.catalog_returns)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |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 ea90224..4cc1702 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
@@ -1,58 +1,57 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |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|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |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 [$$53][$$57]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                              -- STREAM_PROJECT  |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|
+                                    -- HASH_PARTITION_EXCHANGE [$$56, $$55]  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (tpcds.catalog_returns)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$57]  |PARTITIONED|
+                              -- DATASOURCE_SCAN (tpcds.item)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$53][$$57]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STREAM_PROJECT  |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|
+                                    -- HYBRID_HASH_JOIN [$$53][$$57]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |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|
+                                              -- HASH_PARTITION_EXCHANGE [$$56, $$55]  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (tpcds.catalog_returns)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$57]  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (tpcds.item)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$56, $$55]  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (tpcds.catalog_returns)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$57]  |PARTITIONED|
-                                -- DATASOURCE_SCAN (tpcds.item)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- HYBRID_HASH_JOIN [$$53][$$57]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |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|
-                                                -- HASH_PARTITION_EXCHANGE [$$56, $$55]  |PARTITIONED|
-                                                  -- DATASOURCE_SCAN (tpcds.catalog_returns)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |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 cd5de62..9308292 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
@@ -1,43 +1,40 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$l_shipmode(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$135]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$118]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$118][$$124]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- HYBRID_HASH_JOIN [$$126][$$122]  |PARTITIONED|
-                                -- HASH_PARTITION_EXCHANGE [$$126]  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- STREAM_SELECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$l_shipmode(ASC) ]  |PARTITIONED|
+        -- SORT_GROUP_BY[$$135]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$118]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$118][$$124]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$126][$$122]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$126]  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                          -- UNNEST  |UNPARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- UNNEST  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast.plan
index 9908238..3a0c212 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast.plan
@@ -1,43 +1,40 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$l_shipmode(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$131]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$131]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$114]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$114][$$120]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- HYBRID_HASH_JOIN [$$121][$$118]  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- STREAM_SELECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$l_shipmode(ASC) ]  |PARTITIONED|
+        -- SORT_GROUP_BY[$$135]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$118]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$118][$$124]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$125][$$122]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                          -- UNNEST  |UNPARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- UNNEST  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast_ps.plan
index baeda7a..cacd6c1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast_ps.plan
@@ -1,95 +1,90 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$l_shipmode(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$l_shipmode(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- SORT_GROUP_BY[$$131]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- HASH_PARTITION_EXCHANGE [$$131]  |PARTITIONED|
-                          -- SORT_GROUP_BY[$$114]  |PARTITIONED|
-                                  {
-                                    -- AGGREGATE  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                  }
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$l_shipmode(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$l_shipmode(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- SORT_GROUP_BY[$$135]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$118]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$118][$$124]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$125][$$122]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- UNNEST  |UNPARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$114][$$120]  |PARTITIONED|
+                              -- SORT_GROUP_BY[$$135]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
+                                  -- SORT_GROUP_BY[$$118]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- HYBRID_HASH_JOIN [$$121][$$118]  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$118][$$124]  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                               -- STREAM_PROJECT  |PARTITIONED|
-                                                -- STREAM_SELECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                      -- UNNEST  |UNPARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- SORT_GROUP_BY[$$131]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- HASH_PARTITION_EXCHANGE [$$131]  |PARTITIONED|
-                                    -- SORT_GROUP_BY[$$114]  |PARTITIONED|
-                                            {
-                                              -- AGGREGATE  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                            }
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- HYBRID_HASH_JOIN [$$114][$$120]  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- HYBRID_HASH_JOIN [$$121][$$118]  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- STREAM_SELECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- HYBRID_HASH_JOIN [$$125][$$122]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                -- UNNEST  |UNPARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- UNNEST  |UNPARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
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 03a4f96..1ca832e 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
@@ -1,95 +1,90 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$l_shipmode(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$l_shipmode(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- SORT_GROUP_BY[$$135]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
-                          -- SORT_GROUP_BY[$$118]  |PARTITIONED|
-                                  {
-                                    -- AGGREGATE  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                  }
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$l_shipmode(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$l_shipmode(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- SORT_GROUP_BY[$$135]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$118]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$118][$$124]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$126][$$122]  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$126]  |PARTITIONED|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- UNNEST  |UNPARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$118][$$124]  |PARTITIONED|
+                              -- SORT_GROUP_BY[$$135]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
+                                  -- SORT_GROUP_BY[$$118]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- HYBRID_HASH_JOIN [$$126][$$122]  |PARTITIONED|
-                                            -- HASH_PARTITION_EXCHANGE [$$126]  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$118][$$124]  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                               -- STREAM_PROJECT  |PARTITIONED|
-                                                -- STREAM_SELECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                            -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                      -- UNNEST  |UNPARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- SORT_GROUP_BY[$$135]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
-                                    -- SORT_GROUP_BY[$$118]  |PARTITIONED|
-                                            {
-                                              -- AGGREGATE  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                            }
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- HYBRID_HASH_JOIN [$$118][$$124]  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- HYBRID_HASH_JOIN [$$126][$$122]  |PARTITIONED|
-                                                      -- HASH_PARTITION_EXCHANGE [$$126]  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- STREAM_SELECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                      -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- HYBRID_HASH_JOIN [$$126][$$122]  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$126]  |PARTITIONED|
+                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                -- UNNEST  |UNPARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- UNNEST  |UNPARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/union/union_opt_1_1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/union/union_opt_1_1.plan
index bf2376e..fc94531 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/union/union_opt_1_1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/union/union_opt_1_1.plan
@@ -1,47 +1,41 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$122(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$122(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- UNION_ALL  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.onek1.onek1)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$165(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.onek1.onek1_idx)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$126(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$126(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- UNION_ALL  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.onek1.onek1)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$169(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (test.onek1.onek1_idx)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- REPLICATE  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                               -- ASSIGN  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- REPLICATE  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.onek2.onek2)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$168(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.onek2.onek2_idx)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.onek2.onek2)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$172(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (test.onek2.onek2_idx)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- REPLICATE  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/union/union_opt_1_2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/union/union_opt_1_2.plan
index c3dede7..11174a8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/union/union_opt_1_2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/union/union_opt_1_2.plan
@@ -1,48 +1,43 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$116(ASC) ]  |PARTITIONED|
-        -- STABLE_SORT [$$116(ASC)]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$120(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$120(ASC)]  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- UNION_ALL  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.onek1.onek1)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.onek1.onek1)  |PARTITIONED|
+                            -- STABLE_SORT [$$145(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$141(ASC)]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.onek1.onek1_idx)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.onek1.onek1_idx)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- REPLICATE  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.onek2.onek2)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$144(ASC)]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (test.onek2.onek2_idx)  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- REPLICATE  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                 -- ASSIGN  |PARTITIONED|
                                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.onek2.onek2)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$148(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.onek2.onek2_idx)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/union/union_opt_1_3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/union/union_opt_1_3.plan
index 30cd988..e8ee1ea 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/union/union_opt_1_3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/union/union_opt_1_3.plan
@@ -1,49 +1,43 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$60(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- UNION_ALL  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.fbu1.fbu1)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$163(ASC)]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (test.fbu1.fbu1_idx)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$64(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$64(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- UNION_ALL  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.fbu1.fbu1)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$167(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.fbu1.fbu1_idx)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- REPLICATE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                 -- ASSIGN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- REPLICATE  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.fbu2.fbu2)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$166(ASC)]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (test.fbu2.fbu2_idx)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- REPLICATE  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.fbu2.fbu2)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$170(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.fbu2.fbu2_idx)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/union/union_opt_1_4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/union/union_opt_1_4.plan
index 8a1cc02..78c4cc2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/union/union_opt_1_4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/union/union_opt_1_4.plan
@@ -1,50 +1,45 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$112(ASC) ]  |PARTITIONED|
-        -- STABLE_SORT [$$112(ASC)]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$116(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$116(ASC)]  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- UNION_ALL  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (test.fbu1.fbu1)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.fbu1.fbu1)  |PARTITIONED|
+                              -- STABLE_SORT [$$141(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$137(ASC)]  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                      -- BTREE_SEARCH (test.fbu1.fbu1_idx)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (test.fbu1.fbu1_idx)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- REPLICATE  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.fbu2.fbu2)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$140(ASC)]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (test.fbu2.fbu2_idx)  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                               -- REPLICATE  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                   -- ASSIGN  |PARTITIONED|
                                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (test.fbu2.fbu2)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$144(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (test.fbu2.fbu2_idx)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- REPLICATE  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/pg_win/pg_win_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/pg_win/pg_win_01.plan
index 1fee6e7..49d2e01b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/pg_win/pg_win_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/pg_win/pg_win_01.plan
@@ -1,9 +1,8 @@
 -- DISTRIBUTE_RESULT  |LOCAL|
   -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-    -- STREAM_PROJECT  |LOCAL|
-      -- ASSIGN  |LOCAL|
+    -- ASSIGN  |LOCAL|
+      -- AGGREGATE  |LOCAL|
         -- AGGREGATE  |LOCAL|
-          -- AGGREGATE  |LOCAL|
-            -- UNNEST  |UNPARTITIONED|
-              -- STREAM_SELECT  |UNPARTITIONED|
-                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
\ No newline at end of file
+          -- UNNEST  |UNPARTITIONED|
+            -- STREAM_SELECT  |UNPARTITIONED|
+              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_misc/win_misc_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_misc/win_misc_01.plan
index d58eaa1..b4b5e4b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_misc/win_misc_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_misc/win_misc_01.plan
@@ -1,29 +1,23 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
                 -- ASSIGN  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- WINDOW  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$t1.c2(ASC), $$p(ASC), $$n(DESC)]  |PARTITIONED|
-                                  -- HASH_PARTITION_EXCHANGE [$$t1.c2, $$p]  |PARTITIONED|
+                    -- WINDOW  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$t1.c2(ASC), $$p(ASC), $$n(DESC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$t1.c2, $$p]  |PARTITIONED|
+                            -- UNNEST  |PARTITIONED|
+                              -- UNNEST  |PARTITIONED|
+                                -- UNNEST  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
-                                      -- UNNEST  |PARTITIONED|
-                                        -- UNNEST  |PARTITIONED|
-                                          -- UNNEST  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_misc/win_misc_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_misc/win_misc_02.plan
index a7c9cf6..dcb19ce 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_misc/win_misc_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_misc/win_misc_02.plan
@@ -1,23 +1,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$51(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$51(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- WINDOW  |PARTITIONED|
-                        {
-                          -- AGGREGATE  |LOCAL|
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                        }
-                  -- WINDOW_STREAM  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$37(ASC), $$51(ASC)]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$54(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- WINDOW  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- WINDOW_STREAM  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$40(ASC), $$54(ASC)]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$40]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_1.plan
index 871ea8e..eea2af0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_1.plan
@@ -1,11 +1,10 @@
 -- DISTRIBUTE_RESULT  |LOCAL|
   -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-    -- STREAM_PROJECT  |LOCAL|
-      -- ASSIGN  |LOCAL|
-        -- WINDOW  |LOCAL|
-          -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-            -- STABLE_SORT [$$m(ASC), $$t(ASC)]  |LOCAL|
-              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                -- ASSIGN  |UNPARTITIONED|
-                  -- UNNEST  |UNPARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+    -- ASSIGN  |LOCAL|
+      -- WINDOW  |LOCAL|
+        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+          -- STABLE_SORT [$$m(ASC), $$t(ASC)]  |LOCAL|
+            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+              -- ASSIGN  |UNPARTITIONED|
+                -- UNNEST  |UNPARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_10.plan
index a131487..48f9c4b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_10.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_10.plan
@@ -1,15 +1,14 @@
 -- DISTRIBUTE_RESULT  |LOCAL|
   -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-    -- STREAM_PROJECT  |LOCAL|
-      -- ASSIGN  |LOCAL|
-        -- WINDOW  |LOCAL|
-                {
-                  -- AGGREGATE  |LOCAL|
-                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                }
-          -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-            -- STABLE_SORT [$$m(ASC), $$t(ASC)]  |LOCAL|
-              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                -- ASSIGN  |UNPARTITIONED|
-                  -- UNNEST  |UNPARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+    -- ASSIGN  |LOCAL|
+      -- WINDOW  |LOCAL|
+              {
+                -- AGGREGATE  |LOCAL|
+                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+              }
+        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+          -- STABLE_SORT [$$m(ASC), $$t(ASC)]  |LOCAL|
+            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+              -- ASSIGN  |UNPARTITIONED|
+                -- UNNEST  |UNPARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_11.plan
index a131487..48f9c4b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_11.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_11.plan
@@ -1,15 +1,14 @@
 -- DISTRIBUTE_RESULT  |LOCAL|
   -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-    -- STREAM_PROJECT  |LOCAL|
-      -- ASSIGN  |LOCAL|
-        -- WINDOW  |LOCAL|
-                {
-                  -- AGGREGATE  |LOCAL|
-                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                }
-          -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-            -- STABLE_SORT [$$m(ASC), $$t(ASC)]  |LOCAL|
-              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                -- ASSIGN  |UNPARTITIONED|
-                  -- UNNEST  |UNPARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+    -- ASSIGN  |LOCAL|
+      -- WINDOW  |LOCAL|
+              {
+                -- AGGREGATE  |LOCAL|
+                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+              }
+        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+          -- STABLE_SORT [$$m(ASC), $$t(ASC)]  |LOCAL|
+            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+              -- ASSIGN  |UNPARTITIONED|
+                -- UNNEST  |UNPARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_12.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_12.plan
index 77572d4..9540028 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_12.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_12.plan
@@ -1,24 +1,23 @@
 -- DISTRIBUTE_RESULT  |LOCAL|
   -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-    -- STREAM_PROJECT  |LOCAL|
-      -- ASSIGN  |LOCAL|
-        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-          -- STABLE_SORT [$$x(ASC), $$93(ASC)]  |LOCAL|
-            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-              -- STREAM_PROJECT  |LOCAL|
-                -- WINDOW_STREAM  |LOCAL|
-                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                    -- STABLE_SORT [$$x(ASC), $$94(ASC)]  |LOCAL|
-                      -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                        -- PRE_CLUSTERED_GROUP_BY[$$96, $$97]  |LOCAL|
-                                {
+    -- ASSIGN  |LOCAL|
+      -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+        -- STABLE_SORT [$$x(ASC), $$106(ASC)]  |LOCAL|
+          -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+            -- STREAM_PROJECT  |LOCAL|
+              -- WINDOW_STREAM  |LOCAL|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STABLE_SORT [$$x(ASC), $$107(ASC)]  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                      -- PRE_CLUSTERED_GROUP_BY[$$109, $$110]  |LOCAL|
+                              {
+                                -- AGGREGATE  |LOCAL|
                                   -- AGGREGATE  |LOCAL|
-                                    -- AGGREGATE  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
-                          -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                            -- STABLE_SORT [$$96(ASC), $$97(ASC)]  |LOCAL|
-                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                -- ASSIGN  |UNPARTITIONED|
-                                  -- UNNEST  |UNPARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
\ No newline at end of file
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                          -- STABLE_SORT [$$109(ASC), $$110(ASC)]  |LOCAL|
+                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                              -- ASSIGN  |UNPARTITIONED|
+                                -- UNNEST  |UNPARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_2.plan
index a131487..48f9c4b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_2.plan
@@ -1,15 +1,14 @@
 -- DISTRIBUTE_RESULT  |LOCAL|
   -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-    -- STREAM_PROJECT  |LOCAL|
-      -- ASSIGN  |LOCAL|
-        -- WINDOW  |LOCAL|
-                {
-                  -- AGGREGATE  |LOCAL|
-                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                }
-          -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-            -- STABLE_SORT [$$m(ASC), $$t(ASC)]  |LOCAL|
-              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                -- ASSIGN  |UNPARTITIONED|
-                  -- UNNEST  |UNPARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+    -- ASSIGN  |LOCAL|
+      -- WINDOW  |LOCAL|
+              {
+                -- AGGREGATE  |LOCAL|
+                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+              }
+        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+          -- STABLE_SORT [$$m(ASC), $$t(ASC)]  |LOCAL|
+            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+              -- ASSIGN  |UNPARTITIONED|
+                -- UNNEST  |UNPARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_3.plan
index a131487..48f9c4b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_3.plan
@@ -1,15 +1,14 @@
 -- DISTRIBUTE_RESULT  |LOCAL|
   -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-    -- STREAM_PROJECT  |LOCAL|
-      -- ASSIGN  |LOCAL|
-        -- WINDOW  |LOCAL|
-                {
-                  -- AGGREGATE  |LOCAL|
-                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                }
-          -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-            -- STABLE_SORT [$$m(ASC), $$t(ASC)]  |LOCAL|
-              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                -- ASSIGN  |UNPARTITIONED|
-                  -- UNNEST  |UNPARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+    -- ASSIGN  |LOCAL|
+      -- WINDOW  |LOCAL|
+              {
+                -- AGGREGATE  |LOCAL|
+                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+              }
+        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+          -- STABLE_SORT [$$m(ASC), $$t(ASC)]  |LOCAL|
+            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+              -- ASSIGN  |UNPARTITIONED|
+                -- UNNEST  |UNPARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_4.plan
index a131487..48f9c4b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_4.plan
@@ -1,15 +1,14 @@
 -- DISTRIBUTE_RESULT  |LOCAL|
   -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-    -- STREAM_PROJECT  |LOCAL|
-      -- ASSIGN  |LOCAL|
-        -- WINDOW  |LOCAL|
-                {
-                  -- AGGREGATE  |LOCAL|
-                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                }
-          -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-            -- STABLE_SORT [$$m(ASC), $$t(ASC)]  |LOCAL|
-              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                -- ASSIGN  |UNPARTITIONED|
-                  -- UNNEST  |UNPARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+    -- ASSIGN  |LOCAL|
+      -- WINDOW  |LOCAL|
+              {
+                -- AGGREGATE  |LOCAL|
+                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+              }
+        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+          -- STABLE_SORT [$$m(ASC), $$t(ASC)]  |LOCAL|
+            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+              -- ASSIGN  |UNPARTITIONED|
+                -- UNNEST  |UNPARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_5.plan
index 871ea8e..eea2af0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_5.plan
@@ -1,11 +1,10 @@
 -- DISTRIBUTE_RESULT  |LOCAL|
   -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-    -- STREAM_PROJECT  |LOCAL|
-      -- ASSIGN  |LOCAL|
-        -- WINDOW  |LOCAL|
-          -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-            -- STABLE_SORT [$$m(ASC), $$t(ASC)]  |LOCAL|
-              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                -- ASSIGN  |UNPARTITIONED|
-                  -- UNNEST  |UNPARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+    -- ASSIGN  |LOCAL|
+      -- WINDOW  |LOCAL|
+        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+          -- STABLE_SORT [$$m(ASC), $$t(ASC)]  |LOCAL|
+            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+              -- ASSIGN  |UNPARTITIONED|
+                -- UNNEST  |UNPARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_6.plan
index a1e04ad..9c9856b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_6.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_6.plan
@@ -1,12 +1,11 @@
 -- DISTRIBUTE_RESULT  |LOCAL|
   -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-    -- STREAM_PROJECT  |LOCAL|
-      -- ASSIGN  |LOCAL|
-        -- WINDOW  |LOCAL|
-          -- WINDOW_STREAM  |LOCAL|
-            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-              -- STABLE_SORT [$$m(ASC), $$t(ASC)]  |LOCAL|
-                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                  -- ASSIGN  |UNPARTITIONED|
-                    -- UNNEST  |UNPARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+    -- ASSIGN  |LOCAL|
+      -- WINDOW  |LOCAL|
+        -- WINDOW_STREAM  |LOCAL|
+          -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+            -- STABLE_SORT [$$m(ASC), $$t(ASC)]  |LOCAL|
+              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                -- ASSIGN  |UNPARTITIONED|
+                  -- UNNEST  |UNPARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_7.plan
index b111336..f02cdb4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_7.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_7.plan
@@ -1,17 +1,16 @@
 -- DISTRIBUTE_RESULT  |LOCAL|
   -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-    -- STREAM_PROJECT  |LOCAL|
-      -- ASSIGN  |LOCAL|
-        -- STREAM_PROJECT  |LOCAL|
-          -- WINDOW  |LOCAL|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- WINDOW_STREAM  |LOCAL|
-              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                -- STABLE_SORT [$$m(ASC), $$t(ASC)]  |LOCAL|
-                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                    -- ASSIGN  |UNPARTITIONED|
-                      -- UNNEST  |UNPARTITIONED|
-                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+    -- ASSIGN  |LOCAL|
+      -- STREAM_PROJECT  |LOCAL|
+        -- WINDOW  |LOCAL|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- WINDOW_STREAM  |LOCAL|
+            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+              -- STABLE_SORT [$$m(ASC), $$t(ASC)]  |LOCAL|
+                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                  -- ASSIGN  |UNPARTITIONED|
+                    -- UNNEST  |UNPARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_9.plan
index ac920c2..6136376 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_9.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_9.plan
@@ -1,14 +1,13 @@
 -- DISTRIBUTE_RESULT  |LOCAL|
   -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-    -- STREAM_PROJECT  |LOCAL|
-      -- ASSIGN  |LOCAL|
-        -- WINDOW  |LOCAL|
-                {
-                  -- AGGREGATE  |LOCAL|
-                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                }
-          -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-            -- STABLE_SORT [$$x(ASC)]  |LOCAL|
-              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                -- UNNEST  |UNPARTITIONED|
-                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+    -- ASSIGN  |LOCAL|
+      -- WINDOW  |LOCAL|
+              {
+                -- AGGREGATE  |LOCAL|
+                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+              }
+        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+          -- STABLE_SORT [$$x(ASC)]  |LOCAL|
+            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+              -- UNNEST  |UNPARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_02/win_opt_02_1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_02/win_opt_02_1.plan
index 931e417..367e8cb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_02/win_opt_02_1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_02/win_opt_02_1.plan
@@ -1,23 +1,21 @@
 -- DISTRIBUTE_RESULT  |LOCAL|
   -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-    -- STREAM_PROJECT  |LOCAL|
+    -- ASSIGN  |LOCAL|
       -- ASSIGN  |LOCAL|
         -- STREAM_PROJECT  |LOCAL|
-          -- ASSIGN  |LOCAL|
-            -- STREAM_PROJECT  |LOCAL|
-              -- WINDOW  |LOCAL|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- WINDOW  |LOCAL|
-                        {
-                          -- AGGREGATE  |LOCAL|
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                        }
-                  -- WINDOW_STREAM  |LOCAL|
-                    -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                      -- STABLE_SORT [$$r(ASC)]  |LOCAL|
-                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                          -- UNNEST  |UNPARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
\ No newline at end of file
+          -- WINDOW  |LOCAL|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- WINDOW  |LOCAL|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- WINDOW_STREAM  |LOCAL|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STABLE_SORT [$$r(ASC)]  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                      -- UNNEST  |UNPARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ASTERIXDB-2402.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ASTERIXDB-2402.plan
index 33d5a4e..62e50aa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ASTERIXDB-2402.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ASTERIXDB-2402.plan
@@ -29,140 +29,131 @@
                 -- INSERT_DELETE  |PARTITIONED|
                   -- HASH_PARTITION_EXCHANGE [$$214]  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
                         -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$267]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$267]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$267(ASC)]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$267]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                  -- STABLE_SORT [$$267(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$267]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$266][$$237]  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$266][$$237]  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- UNNEST  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- PRE_CLUSTERED_GROUP_BY[$$319]  |PARTITIONED|
-                                                                  {
-                                                                    -- AGGREGATE  |LOCAL|
-                                                                      -- MICRO_PRE_CLUSTERED_GROUP_BY[$$321, $$322]  |LOCAL|
-                                                                              {
-                                                                                -- AGGREGATE  |LOCAL|
-                                                                                  -- STREAM_SELECT  |LOCAL|
-                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                              }
-                                                                        -- STREAM_SELECT  |LOCAL|
-                                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                  }
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- STABLE_SORT [$$319(ASC), $$321(ASC), $$322(ASC)]  |PARTITIONED|
-                                                                -- HASH_PARTITION_EXCHANGE [$$319]  |PARTITIONED|
-                                                                  -- UNION_ALL  |PARTITIONED|
+                                              -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                -- UNNEST  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- PRE_CLUSTERED_GROUP_BY[$$319]  |PARTITIONED|
+                                                            {
+                                                              -- AGGREGATE  |LOCAL|
+                                                                -- MICRO_PRE_CLUSTERED_GROUP_BY[$$321, $$322]  |LOCAL|
+                                                                        {
+                                                                          -- AGGREGATE  |LOCAL|
+                                                                            -- STREAM_SELECT  |LOCAL|
+                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                        }
+                                                                  -- STREAM_SELECT  |LOCAL|
+                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                            }
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STABLE_SORT [$$319(ASC), $$321(ASC), $$322(ASC)]  |PARTITIONED|
+                                                          -- HASH_PARTITION_EXCHANGE [$$319]  |PARTITIONED|
+                                                            -- UNION_ALL  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                      -- BTREE_SEARCH (channels.Shelters.Shelters)  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                           -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- BTREE_SEARCH (channels.Shelters.Shelters)  |PARTITIONED|
-                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- SPLIT  |PARTITIONED|
-                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                -- RTREE_SEARCH (channels.Shelters.s_location)  |PARTITIONED|
-                                                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                        -- NESTED_LOOP  |PARTITIONED|
-                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                            -- NESTED_LOOP  |PARTITIONED|
-                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                    -- DATASOURCE_SCAN (channels.EmergenciesNearMeChannelChannelSubscriptions)  |PARTITIONED|
-                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                                                  -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                        -- DATASOURCE_SCAN (channels.Reports)  |PARTITIONED|
-                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                            -- ASSIGN  |PARTITIONED|
-                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                -- DATASOURCE_SCAN (channels.UserLocations)  |PARTITIONED|
-                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        -- STREAM_SELECT  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- SPLIT  |PARTITIONED|
                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- SPLIT  |PARTITIONED|
+                                                                                  -- STREAM_PROJECT  |PARTITIONED|
                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                          -- RTREE_SEARCH (channels.Shelters.s_location)  |PARTITIONED|
-                                                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                      -- RTREE_SEARCH (channels.Shelters.s_location)  |PARTITIONED|
+                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- NESTED_LOOP  |PARTITIONED|
                                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                   -- NESTED_LOOP  |PARTITIONED|
                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                      -- NESTED_LOOP  |PARTITIONED|
-                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          -- ASSIGN  |PARTITIONED|
-                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                              -- DATASOURCE_SCAN (channels.EmergenciesNearMeChannelChannelSubscriptions)  |PARTITIONED|
-                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                          -- ASSIGN  |PARTITIONED|
-                                                                                                            -- STREAM_SELECT  |PARTITIONED|
-                                                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                  -- DATASOURCE_SCAN (channels.Reports)  |PARTITIONED|
-                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
                                                                                                       -- ASSIGN  |PARTITIONED|
                                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          -- DATASOURCE_SCAN (channels.UserLocations)  |PARTITIONED|
+                                                                                                          -- DATASOURCE_SCAN (channels.EmergenciesNearMeChannelChannelSubscriptions)  |PARTITIONED|
                                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$237]  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- HYBRID_HASH_JOIN [$$248, $$250][$$239, $$240]  |PARTITIONED|
-                                                          -- HASH_PARTITION_EXCHANGE [$$248, $$250]  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (channels.EmergenciesNearMeChannelBrokerSubscriptions)  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                          -- HASH_PARTITION_EXCHANGE [$$239, $$240]  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (channels.Broker)  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- DATASOURCE_SCAN (channels.Reports)  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                      -- DATASOURCE_SCAN (channels.UserLocations)  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- SPLIT  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- RTREE_SEARCH (channels.Shelters.s_location)  |PARTITIONED|
+                                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- NESTED_LOOP  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- NESTED_LOOP  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- DATASOURCE_SCAN (channels.EmergenciesNearMeChannelChannelSubscriptions)  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- DATASOURCE_SCAN (channels.Reports)  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- DATASOURCE_SCAN (channels.UserLocations)  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$237]  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$248, $$250][$$239, $$240]  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$248, $$250]  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (channels.EmergenciesNearMeChannelBrokerSubscriptions)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$239, $$240]  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (channels.Broker)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.10.plan
index 9ea1780..665d725 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.10.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.10.plan
@@ -1,35 +1,33 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                        -- MICRO_STABLE_SORT [$$67(ASC)]  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
-                  -- NESTED_LOOP  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- NESTED_LOOP  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                      -- MICRO_STABLE_SORT [$$72(ASC)]  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- NESTED_LOOP  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                          -- UNNEST  |UNPARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- UNNEST  |UNPARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.11.plan
index ebfcfc2..55a8c3a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.11.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.11.plan
@@ -1,43 +1,40 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                        -- MICRO_STABLE_SORT [$$67(ASC)]  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                        -- MICRO_STABLE_SORT [$$72(ASC)]  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
-                  -- NESTED_LOOP  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- NESTED_LOOP  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                      -- MICRO_STABLE_SORT [$$72(ASC)]  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                      -- MICRO_STABLE_SORT [$$77(ASC)]  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- NESTED_LOOP  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                          -- UNNEST  |UNPARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
                         -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- UNNEST  |UNPARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.12.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.12.plan
index a8f346f..523e336 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.12.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.12.plan
@@ -1,39 +1,37 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                        -- MICRO_STABLE_SORT [$$71(ASC)]  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
-                  -- NESTED_LOOP  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- NESTED_LOOP  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                      -- MICRO_STABLE_SORT [$$76(ASC)]  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- NESTED_LOOP  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                          -- UNNEST  |UNPARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- UNNEST  |UNPARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.13.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.13.plan
index 439b1d4..aefdb9a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.13.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.13.plan
@@ -1,57 +1,53 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                        -- MICRO_STABLE_SORT [$$82(ASC)]  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                        -- MICRO_STABLE_SORT [$$92(ASC)]  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
-                  -- NESTED_LOOP  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- NESTED_LOOP  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- NESTED_LOOP  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                      -- MICRO_STABLE_SORT [$$88(ASC)]  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                      -- MICRO_STABLE_SORT [$$98(ASC)]  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- NESTED_LOOP  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- NESTED_LOOP  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                              -- UNNEST  |UNPARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
                         -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- DATASOURCE_SCAN (test.d3)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- UNNEST  |UNPARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.14.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.14.plan
index c57b6f9..c522bc1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.14.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.14.plan
@@ -1,63 +1,59 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                        -- MICRO_STABLE_SORT [$$106(ASC)]  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                        -- MICRO_STABLE_SORT [$$116(ASC)]  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                        -- MICRO_STABLE_SORT [$$111(ASC)]  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
-                  -- NESTED_LOOP  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- NESTED_LOOP  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- NESTED_LOOP  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                      -- MICRO_STABLE_SORT [$$112(ASC)]  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                      -- MICRO_STABLE_SORT [$$122(ASC)]  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                      -- MICRO_STABLE_SORT [$$117(ASC)]  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- NESTED_LOOP  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- NESTED_LOOP  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                              -- UNNEST  |UNPARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
                         -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- DATASOURCE_SCAN (test.d3)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- UNNEST  |UNPARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.9.plan
index 8802ff5..979d032 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.9.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.9.plan
@@ -1,35 +1,33 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                        -- MICRO_STABLE_SORT [$$63(ASC)]  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
-                  -- NESTED_LOOP  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- NESTED_LOOP  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                      -- MICRO_STABLE_SORT [$$68(ASC)]  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- NESTED_LOOP  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                          -- UNNEST  |UNPARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- UNNEST  |UNPARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query1.plan
index d69738a..a7f9021 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query1.plan
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- UNNEST  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (TestDataverse.Dataset1)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (TestDataverse.Dataset1)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query2.plan
index d69738a..a7f9021 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query2.plan
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- UNNEST  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (TestDataverse.Dataset1)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (TestDataverse.Dataset1)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query3.plan
index 52f7d02..b14812e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query3.plan
@@ -1,18 +1,15 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- UNNEST  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- UNNEST  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (TestDataverse.Dataset1)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (TestDataverse.Dataset1)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query4.plan
index 6d57ef0..ab210b2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query4.plan
@@ -1,22 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- ASSIGN  |LOCAL|
-                              -- UNNEST  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- ASSIGN  |LOCAL|
+                          -- UNNEST  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (TestDataverse.Dataset1)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (TestDataverse.Dataset1)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query5.plan
index 9f3e9a0..6a47d67 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query5.plan
@@ -1,31 +1,29 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- SUBPLAN  |LOCAL|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- ASSIGN  |LOCAL|
+                                        -- UNNEST  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
                           -- STREAM_SELECT  |LOCAL|
-                            -- SUBPLAN  |LOCAL|
-                                    {
-                                      -- AGGREGATE  |LOCAL|
-                                        -- STREAM_SELECT  |LOCAL|
-                                          -- ASSIGN  |LOCAL|
-                                            -- UNNEST  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                    }
-                              -- STREAM_SELECT  |LOCAL|
-                                -- ASSIGN  |LOCAL|
-                                  -- UNNEST  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |LOCAL|
+                              -- UNNEST  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (TestDataverse.Dataset1)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (TestDataverse.Dataset1)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query8.plan
index ea7ceef..9bf272a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query8.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query8.plan
@@ -1,38 +1,36 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- ASSIGN  |LOCAL|
-                              -- UNNEST  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- ASSIGN  |LOCAL|
+                          -- UNNEST  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                              -- STABLE_SORT [$$77(ASC), $$78(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$74(ASC), $$75(ASC)]  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                              -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- RUNNING_AGGREGATE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (TestDataverse.Dataset2)  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (TestDataverse.Dataset2)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query9.plan
index b47e122..ab7ffda 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query9.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/atomic-and-array-queries/query9.plan
@@ -1,47 +1,45 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- SUBPLAN  |LOCAL|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- ASSIGN  |LOCAL|
+                                        -- UNNEST  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
                           -- STREAM_SELECT  |LOCAL|
-                            -- SUBPLAN  |LOCAL|
-                                    {
-                                      -- AGGREGATE  |LOCAL|
-                                        -- STREAM_SELECT  |LOCAL|
-                                          -- ASSIGN  |LOCAL|
-                                            -- UNNEST  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                    }
-                              -- STREAM_SELECT  |LOCAL|
-                                -- ASSIGN  |LOCAL|
-                                  -- UNNEST  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |LOCAL|
+                              -- UNNEST  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                              -- STABLE_SORT [$$108(ASC), $$109(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$104(ASC), $$105(ASC)]  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                              -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- RUNNING_AGGREGATE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (TestDataverse.Dataset2)  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (TestDataverse.Dataset2)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/multiple-indexes/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/multiple-indexes/query1.plan
index 498ee33..f5099d7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/multiple-indexes/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/multiple-indexes/query1.plan
@@ -1,21 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- ASSIGN  |LOCAL|
-                              -- UNNEST  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- ASSIGN  |LOCAL|
+                          -- UNNEST  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.KSI)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.KSI)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/multiple-indexes/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/multiple-indexes/query2.plan
index 498ee33..f5099d7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/multiple-indexes/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/multiple-indexes/query2.plan
@@ -1,21 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- ASSIGN  |LOCAL|
-                              -- UNNEST  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- ASSIGN  |LOCAL|
+                          -- UNNEST  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.KSI)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.KSI)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/multiple-quantifiers/query7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/multiple-quantifiers/query7.plan
index 9e577d4..306ff7f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/multiple-quantifiers/query7.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/multiple-quantifiers/query7.plan
@@ -1,22 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- ASSIGN  |LOCAL|
+                        -- UNNEST  |LOCAL|
                           -- ASSIGN  |LOCAL|
                             -- UNNEST  |LOCAL|
-                              -- ASSIGN  |LOCAL|
-                                -- UNNEST  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (TestDataverse.Dataset1)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (TestDataverse.Dataset1)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-1/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-1/query1.plan
index 1abd71a..6208814 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-1/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-1/query1.plan
@@ -1,21 +1,18 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- UNNEST  |LOCAL|
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- UNNEST  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-1/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-1/query2.plan
index 1658a86..a931a58 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-1/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-1/query2.plan
@@ -1,23 +1,21 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- UNNEST  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- UNNEST  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-1/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-1/query4.plan
index 2d561f4..48dfd6c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-1/query4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-1/query4.plan
@@ -1,22 +1,20 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- UNNEST  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- UNNEST  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-2/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-2/query1.plan
index 1abd71a..6208814 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-2/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-2/query1.plan
@@ -1,21 +1,18 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- UNNEST  |LOCAL|
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- UNNEST  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-2/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-2/query2.plan
index 1658a86..a931a58 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-2/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-2/query2.plan
@@ -1,23 +1,21 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- UNNEST  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- STREAM_SELECT  |LOCAL|
+                              -- UNNEST  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-2/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-2/query3.plan
index 2d561f4..48dfd6c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-2/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-2/query3.plan
@@ -1,22 +1,20 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- SUBPLAN  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- UNNEST  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- SUBPLAN  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- UNNEST  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-3/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-3/query1.plan
index c5a8718..507cf3e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-3/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-3/query1.plan
@@ -1,22 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- ASSIGN  |LOCAL|
-                              -- UNNEST  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- ASSIGN  |LOCAL|
+                          -- UNNEST  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-3/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-3/query2.plan
index c5a8718..507cf3e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-3/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-3/query2.plan
@@ -1,22 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- ASSIGN  |LOCAL|
-                              -- UNNEST  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- ASSIGN  |LOCAL|
+                          -- UNNEST  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-3/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-3/query3.plan
index 7e503bb..ff93394 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-3/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-quantified-queries/use-case-3/query3.plan
@@ -1,21 +1,18 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- SUBPLAN  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- ASSIGN  |LOCAL|
-                            -- UNNEST  |LOCAL|
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- SUBPLAN  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- ASSIGN  |LOCAL|
+                        -- UNNEST  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-1/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-1/query1.plan
index 834d7d2..51d0004 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-1/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-1/query1.plan
@@ -1,14 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-1/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-1/query2.plan
index d18b9b4..1ff305e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-1/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-1/query2.plan
@@ -1,17 +1,14 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-2/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-2/query1.plan
index 834d7d2..51d0004 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-2/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-2/query1.plan
@@ -1,14 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-2/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-2/query2.plan
index d18b9b4..1ff305e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-2/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-2/query2.plan
@@ -1,17 +1,14 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-3/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-3/query1.plan
index 834d7d2..51d0004 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-3/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-3/query1.plan
@@ -1,14 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-3/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-3/query2.plan
index 5e46297..a2530d6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-3/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-3/query2.plan
@@ -1,19 +1,15 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- UNNEST  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- UNNEST  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-3/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-3/query3.plan
index 834d7d2..51d0004 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-3/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-3/query3.plan
@@ -1,14 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-4/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-4/query1.plan
index 7c44281..e1cb4ae 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-4/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-4/query1.plan
@@ -1,18 +1,13 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- UNNEST  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- UNNEST  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-4/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-4/query2.plan
index b191fe2..0f29817 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-4/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/use-case-4/query2.plan
@@ -1,21 +1,16 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- UNNEST  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.plan
index 834d7d2..51d0004 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.plan
@@ -1,14 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.plan
index d18b9b4..1ff305e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.plan
@@ -1,17 +1,14 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-pk/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-pk/query1.plan
index 070c67b..51d0004 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-pk/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-pk/query1.plan
@@ -1,15 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-pk/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-pk/query2.plan
index d18b9b4..1ff305e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-pk/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-pk/query2.plan
@@ -1,17 +1,14 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-sk/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-sk/query1.plan
index e92baf8..be29177 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-sk/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-sk/query1.plan
@@ -1,16 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
           -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-sk/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-sk/query2.plan
index 5e46297..a2530d6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-sk/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-composite-sk/query2.plan
@@ -1,19 +1,15 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- UNNEST  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- UNNEST  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-filter-fields/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-filter-fields/query1.plan
index a224a0e..504456d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-filter-fields/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-filter-fields/query1.plan
@@ -1,17 +1,13 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
           -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-filter-fields/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-filter-fields/query2.plan
index d18b9b4..1ff305e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-filter-fields/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/closed/with-filter-fields/query2.plan
@@ -1,17 +1,14 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query1.plan
index 30d30c6..9e0c394 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query1.plan
@@ -1,13 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- UNNEST  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (TestDataverse.TestDataset)  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- DATASOURCE_SCAN (TestDataverse.TestDataset)  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query2.plan
index 30d30c6..9e0c394 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query2.plan
@@ -1,13 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- UNNEST  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (TestDataverse.TestDataset)  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- DATASOURCE_SCAN (TestDataverse.TestDataset)  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query3.plan
index 4cbcbbd..aa33226 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query3.plan
@@ -1,16 +1,13 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- UNNEST  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (TestDataverse.TestDataset)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (TestDataverse.TestDataset)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query4.plan
index 4cbcbbd..aa33226 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query4.plan
@@ -1,16 +1,13 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- UNNEST  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (TestDataverse.TestDataset)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (TestDataverse.TestDataset)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query5.plan
index 30d30c6..9e0c394 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query5.plan
@@ -1,13 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- UNNEST  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (TestDataverse.TestDataset)  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- DATASOURCE_SCAN (TestDataverse.TestDataset)  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query6.plan
index 30d30c6..9e0c394 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query6.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query6.plan
@@ -1,13 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- UNNEST  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (TestDataverse.TestDataset)  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- DATASOURCE_SCAN (TestDataverse.TestDataset)  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query7.plan
index 4cbcbbd..aa33226 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query7.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query7.plan
@@ -1,16 +1,13 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- UNNEST  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (TestDataverse.TestDataset)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (TestDataverse.TestDataset)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query8.plan
index 4cbcbbd..aa33226 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query8.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/complex-structures/query8.plan
@@ -1,16 +1,13 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- UNNEST  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (TestDataverse.TestDataset)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (TestDataverse.TestDataset)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/multiple-indexes/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/multiple-indexes/query1.plan
index 8a0bc58..0df8d5a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/multiple-indexes/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/multiple-indexes/query1.plan
@@ -1,13 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- UNNEST  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (test.KSI)  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- DATASOURCE_SCAN (test.KSI)  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/multiple-indexes/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/multiple-indexes/query2.plan
index 8a0bc58..0df8d5a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/multiple-indexes/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/multiple-indexes/query2.plan
@@ -1,13 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- UNNEST  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (test.KSI)  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- DATASOURCE_SCAN (test.KSI)  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-1/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-1/query1.plan
index 834d7d2..51d0004 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-1/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-1/query1.plan
@@ -1,14 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-1/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-1/query2.plan
index d18b9b4..1ff305e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-1/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-1/query2.plan
@@ -1,17 +1,14 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-2/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-2/query1.plan
index 834d7d2..51d0004 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-2/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-2/query1.plan
@@ -1,14 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-2/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-2/query2.plan
index d18b9b4..1ff305e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-2/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-2/query2.plan
@@ -1,17 +1,14 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-3/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-3/query1.plan
index 834d7d2..51d0004 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-3/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-3/query1.plan
@@ -1,14 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-3/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-3/query2.plan
index 5e46297..a2530d6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-3/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-3/query2.plan
@@ -1,19 +1,15 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- UNNEST  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- UNNEST  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-3/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-3/query3.plan
index 834d7d2..51d0004 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-3/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-3/query3.plan
@@ -1,14 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-4/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-4/query1.plan
index 7c44281..e1cb4ae 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-4/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-4/query1.plan
@@ -1,18 +1,13 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- UNNEST  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- UNNEST  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-4/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-4/query2.plan
index b191fe2..0f29817 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-4/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/use-case-4/query2.plan
@@ -1,21 +1,16 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- UNNEST  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/with-3-level-record-path/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/with-3-level-record-path/query1.plan
index 834d7d2..51d0004 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/with-3-level-record-path/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/with-3-level-record-path/query1.plan
@@ -1,14 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/with-3-level-record-path/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/with-3-level-record-path/query2.plan
index d18b9b4..1ff305e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/with-3-level-record-path/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/with-3-level-record-path/query2.plan
@@ -1,17 +1,14 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- UNNEST  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- UNNEST  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/with-composite-sk/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/with-composite-sk/query1.plan
index e92baf8..be29177 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/with-composite-sk/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/array-index/select-unnest-queries/open/with-composite-sk/query1.plan
@@ -1,16 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- UNNEST  |PARTITIONED|
           -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- UNNEST  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (TestYelp.YelpCheckin)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-3.plan
index 79492ff..4e9bc29 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-3.plan
@@ -2,24 +2,22 @@
   -- SORT_MERGE_EXCHANGE [$$39(ASC) ]  |PARTITIONED|
     -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk2.tenk2)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.tenk2.idx_tenk2_1k_2k)  |PARTITIONED|
-                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk2.tenk2)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.tenk2.idx_tenk2_1k_2k)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (test.tenk1.tenk1)  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (test.tenk1.tenk1)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-4.plan
index 29d7941..acfe0c3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-4.plan
@@ -6,19 +6,17 @@
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- HYBRID_HASH_JOIN [$$36][$$37]  |PARTITIONED|
               -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (test.tenk1.tenk1)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.tenk2)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.tenk1.tenk1)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.tenk2)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-6.plan
index b6b4ba0..2b707e9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-6.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-6.plan
@@ -1,24 +1,22 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$37(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$37(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$39(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$35][$$34]  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.tenk2)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$36]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.tenk2)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (test.tenk1.tenk1)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (test.tenk1.tenk1)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/secondary-equi-join_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/secondary-equi-join_04.plan
index df3edba..04756e9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/secondary-equi-join_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/secondary-equi-join_04.plan
@@ -1,32 +1,31 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$45][$$49]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (test.testdst3.testdst3)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.testdst3.sec3_Idx)  |PARTITIONED|
-                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (test.testdst2)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.testdst)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- HYBRID_HASH_JOIN [$$45][$$49]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (test.testdst3.testdst3)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.testdst3.sec3_Idx)  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.testdst2)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- BROADCAST_EXCHANGE  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (test.testdst)  |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/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 0e5a3a7..b32ef1c 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
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$22(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.employee)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.employee)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |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 c91805c..311e202 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
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$61(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.ds2)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.ds2)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |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 62d8cf1..9e04d8e 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
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- 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_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$50(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.ds5)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.ds5)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |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 62d8cf1..9e04d8e 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
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- 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_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$50(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.ds5)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.ds5)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |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 62d8cf1..9e04d8e 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
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- 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_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$50(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.ds5)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.ds5)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |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 607e59c..c488edd 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
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- 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_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$50(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.ds6)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.ds6)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |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 607e59c..c488edd 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
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- 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_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$50(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.ds6)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.ds6)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |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 607e59c..c488edd 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
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- 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_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$50(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.ds6)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.ds6)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |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 37bd6b8..161c4d4 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
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$60(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.ds7)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.ds7)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |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 37bd6b8..161c4d4 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
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$60(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.ds7)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.ds7)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |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 12adbc6..12e67c6 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
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.ds7)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.ds7)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |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 37bd6b8..161c4d4 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
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$60(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.ds7)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.ds7)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/disjunctive-predicate/disjunctive-predicate-1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/disjunctive-predicate/disjunctive-predicate-1.plan
index 61695a3..2ebdb2e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/disjunctive-predicate/disjunctive-predicate-1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/disjunctive-predicate/disjunctive-predicate-1.plan
@@ -2,11 +2,9 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- HYBRID_HASH_JOIN [$$20][$$21]  |PARTITIONED|
+        -- BTREE_SEARCH (test.TestSet.TestSet)  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- DATASOURCE_SCAN (test.TestSet)  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-          -- BROADCAST_EXCHANGE  |PARTITIONED|
-            -- UNNEST  |UNPARTITIONED|
-              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+            -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+                -- UNNEST  |UNPARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/disjunctive-predicate/disjunctive-predicate-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/disjunctive-predicate/disjunctive-predicate-2.plan
index 318f7d5..642109c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/disjunctive-predicate/disjunctive-predicate-2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/disjunctive-predicate/disjunctive-predicate-2.plan
@@ -1,15 +1,15 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- HYBRID_HASH_JOIN [$$19][$$21]  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- BTREE_SEARCH (test.TestSet.TestSet)  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- DATASOURCE_SCAN (test.TestSet)  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-          -- BROADCAST_EXCHANGE  |PARTITIONED|
-            -- UNNEST  |UNPARTITIONED|
-              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                      -- BTREE_SEARCH (test.TestSet.TestSetIndex)  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- UNNEST  |UNPARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-10.plan
index c266c21..da99423 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-10.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-10.plan
@@ -1,12 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$19(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (test.tenk)  |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/hints-skip-index/hints-skip-index-11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-11.plan
index a2a9642..88609bd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-11.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-11.plan
@@ -1,18 +1,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$20][$$22]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$21][$$23]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
               -- BROADCAST_EXCHANGE  |PARTITIONED|
                 -- UNNEST  |UNPARTITIONED|
                   -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-12.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-12.plan
index a2a9642..82a6ab3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-12.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-12.plan
@@ -1,18 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$20][$$22]  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                -- UNNEST  |UNPARTITIONED|
-                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                            -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |UNPARTITIONED|
+                                  -- UNNEST  |UNPARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-13.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-13.plan
index c266c21..da99423 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-13.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-13.plan
@@ -1,12 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$19(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (test.tenk)  |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/hints-skip-index/hints-skip-index-4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-4.plan
index c266c21..da99423 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-4.plan
@@ -1,12 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$19(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (test.tenk)  |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/hints-skip-index/hints-skip-index-6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-6.plan
index 86e54c9..fc75e19 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-6.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-6.plan
@@ -1,12 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (test.tenk)  |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/hints-skip-index/hints-skip-index-7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-7.plan
index 86e54c9..fc75e19 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-7.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-7.plan
@@ -1,12 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (test.tenk)  |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/hints-skip-index/hints-skip-index-8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-8.plan
index 86e54c9..fc75e19 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-8.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-8.plan
@@ -1,12 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (test.tenk)  |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/hints-skip-index/hints-skip-index-9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-9.plan
index c266c21..da99423 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-9.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-9.plan
@@ -1,12 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$19(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (test.tenk)  |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/hints-use-index/hints-use-index-10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-10.plan
index de64076..da99423 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-10.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-10.plan
@@ -2,11 +2,10 @@
   -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
     -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (test.tenk)  |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/hints-use-index/hints-use-index-11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-11.plan
index de64076..da99423 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-11.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-11.plan
@@ -2,11 +2,10 @@
   -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
     -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (test.tenk)  |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/hints-use-index/hints-use-index-12.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-12.plan
index 2afa5dd..fc75e19 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-12.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-12.plan
@@ -2,11 +2,10 @@
   -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
     -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (test.tenk)  |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/hints-use-index/hints-use-index-13.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-13.plan
index de64076..da99423 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-13.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-13.plan
@@ -2,11 +2,10 @@
   -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
     -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (test.tenk)  |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/hints-use-index/hints-use-index-14.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-14.plan
index de64076..da99423 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-14.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-14.plan
@@ -2,11 +2,10 @@
   -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
     -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (test.tenk)  |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/hints-use-index/hints-use-index-15.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-15.plan
index 2afa5dd..fc75e19 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-15.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-15.plan
@@ -2,11 +2,10 @@
   -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
     -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (test.tenk)  |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/hints-use-index/hints-use-index-16.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-16.plan
index 2afa5dd..fc75e19 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-16.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-16.plan
@@ -2,11 +2,10 @@
   -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
     -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (test.tenk)  |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/hints-use-index/hints-use-index-17.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-17.plan
index 6e732ed..ad64bb8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-17.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-17.plan
@@ -2,18 +2,17 @@
   -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
     -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-18.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-18.plan
index c266c21..da99423 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-18.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-18.plan
@@ -1,12 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$19(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (test.tenk)  |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/hints-use-index/hints-use-index-19.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-19.plan
index c266c21..da99423 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-19.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-19.plan
@@ -1,12 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$19(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (test.tenk)  |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/hints-use-index/hints-use-index-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-3.plan
index 9643deb..cfff6e8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-3.plan
@@ -2,18 +2,17 @@
   -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
     -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-4.plan
index cf09754..9a52414 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-4.plan
@@ -2,36 +2,34 @@
   -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
     -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INTERSECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.tenk.idx_1k)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- INTERSECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.idx_1k)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.tenk.idx_2k)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- REPLICATE  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.idx_2k)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-5.plan
index 13375cb..5e57a22 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-5.plan
@@ -2,36 +2,34 @@
   -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
     -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INTERSECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- INTERSECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.tenk.idx_5k_10k)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- REPLICATE  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.idx_5k_10k)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-7.plan
index 00ba0d7..797e5fa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-7.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-7.plan
@@ -2,18 +2,17 @@
   -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
     -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-8.plan
index 7e83ccb..e32ac40 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-8.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-8.plan
@@ -2,29 +2,28 @@
   -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
     -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INTERSECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.tenk.idx_2k_5k)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- INTERSECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.idx_2k_5k)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-9.plan
index 7e83ccb..e32ac40 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-9.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-use-index/hints-use-index-9.plan
@@ -2,29 +2,28 @@
   -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
     -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INTERSECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- BTREE_SEARCH (test.tenk.idx_2k_5k)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- INTERSECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.idx_1k_2k)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.idx_2k_5k)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/intersection-misc/intersection-misc-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/intersection-misc/intersection-misc-01.plan
index ee0f744..33ccc2e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/intersection-misc/intersection-misc-01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/intersection-misc/intersection-misc-01.plan
@@ -1,106 +1,96 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$$142(ASC), $$132(ASC), $$144(ASC) ]  |PARTITIONED|
-            -- STABLE_SORT [$$142(ASC), $$132(ASC), $$144(ASC)]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- HYBRID_HASH_JOIN [$$137][$$144]  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$142][$$136]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$142]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$142(ASC), $$132(ASC), $$144(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$142(ASC), $$132(ASC), $$144(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$137][$$144]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$142][$$136]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$142]  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
-                                    -- STREAM_SELECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- PRE_CLUSTERED_GROUP_BY[$$130]  |PARTITIONED|
+                                              {
+                                                -- AGGREGATE  |LOCAL|
+                                                  -- STREAM_SELECT  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                              }
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STABLE_SORT [$$130(ASC)]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- HYBRID_HASH_JOIN [$$139][$$85]  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$139]  |PARTITIONED|
+                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- REPLICATE  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$136]  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$144]  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$133]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$133(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- PRE_CLUSTERED_GROUP_BY[$$130]  |PARTITIONED|
-                                                  {
-                                                    -- AGGREGATE  |LOCAL|
-                                                      -- STREAM_SELECT  |LOCAL|
-                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                  }
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STABLE_SORT [$$130(ASC)]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$140][$$104]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$140]  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- HYBRID_HASH_JOIN [$$139][$$85]  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$139]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- STREAM_SELECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- REPLICATE  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- REPLICATE  |PARTITIONED|
-                                                            -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$136]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- REPLICATE  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$144]  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$133]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$133(ASC)]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$140][$$104]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$140]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- STREAM_SELECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- REPLICATE  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
                                                       -- ASSIGN  |PARTITIONED|
                                                         -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- REPLICATE  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (test.d)  |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 [$$85]  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (test.c)  |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/intersection-misc/intersection-misc-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/intersection-misc/intersection-misc-02.plan
index ee0f744..33ccc2e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/intersection-misc/intersection-misc-02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/intersection-misc/intersection-misc-02.plan
@@ -1,106 +1,96 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$$142(ASC), $$132(ASC), $$144(ASC) ]  |PARTITIONED|
-            -- STABLE_SORT [$$142(ASC), $$132(ASC), $$144(ASC)]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- HYBRID_HASH_JOIN [$$137][$$144]  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$142][$$136]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$142]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$142(ASC), $$132(ASC), $$144(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$142(ASC), $$132(ASC), $$144(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$137][$$144]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$142][$$136]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$142]  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
-                                    -- STREAM_SELECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- PRE_CLUSTERED_GROUP_BY[$$130]  |PARTITIONED|
+                                              {
+                                                -- AGGREGATE  |LOCAL|
+                                                  -- STREAM_SELECT  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                              }
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STABLE_SORT [$$130(ASC)]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- HYBRID_HASH_JOIN [$$139][$$85]  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$139]  |PARTITIONED|
+                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- REPLICATE  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$136]  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$144]  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$133]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$133(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- PRE_CLUSTERED_GROUP_BY[$$130]  |PARTITIONED|
-                                                  {
-                                                    -- AGGREGATE  |LOCAL|
-                                                      -- STREAM_SELECT  |LOCAL|
-                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                  }
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STABLE_SORT [$$130(ASC)]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$140][$$104]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$140]  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- HYBRID_HASH_JOIN [$$139][$$85]  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$139]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- STREAM_SELECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- REPLICATE  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- REPLICATE  |PARTITIONED|
-                                                            -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$136]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- REPLICATE  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$144]  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$133]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$133(ASC)]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$140][$$104]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$140]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- STREAM_SELECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- REPLICATE  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
                                                       -- ASSIGN  |PARTITIONED|
                                                         -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- REPLICATE  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (test.d)  |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 [$$85]  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (test.c)  |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/intersection-misc/intersection-misc-03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/intersection-misc/intersection-misc-03.plan
index ee0f744..33ccc2e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/intersection-misc/intersection-misc-03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/intersection-misc/intersection-misc-03.plan
@@ -1,106 +1,96 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$$142(ASC), $$132(ASC), $$144(ASC) ]  |PARTITIONED|
-            -- STABLE_SORT [$$142(ASC), $$132(ASC), $$144(ASC)]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- HYBRID_HASH_JOIN [$$137][$$144]  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$142][$$136]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$142]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$142(ASC), $$132(ASC), $$144(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$142(ASC), $$132(ASC), $$144(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$137][$$144]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$142][$$136]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$142]  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
-                                    -- STREAM_SELECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- PRE_CLUSTERED_GROUP_BY[$$130]  |PARTITIONED|
+                                              {
+                                                -- AGGREGATE  |LOCAL|
+                                                  -- STREAM_SELECT  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                              }
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STABLE_SORT [$$130(ASC)]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- HYBRID_HASH_JOIN [$$139][$$85]  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$139]  |PARTITIONED|
+                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- REPLICATE  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$136]  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$144]  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$133]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$133(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- PRE_CLUSTERED_GROUP_BY[$$130]  |PARTITIONED|
-                                                  {
-                                                    -- AGGREGATE  |LOCAL|
-                                                      -- STREAM_SELECT  |LOCAL|
-                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                  }
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STABLE_SORT [$$130(ASC)]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$140][$$104]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$140]  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- HYBRID_HASH_JOIN [$$139][$$85]  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$139]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- STREAM_SELECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- REPLICATE  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- REPLICATE  |PARTITIONED|
-                                                            -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$136]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- REPLICATE  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$144]  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$133]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$133(ASC)]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (test.d)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$140][$$104]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$140]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- STREAM_SELECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- REPLICATE  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$85]  |PARTITIONED|
                                                       -- ASSIGN  |PARTITIONED|
                                                         -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- REPLICATE  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (test.d)  |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 [$$85]  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (test.c)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (test.c)  |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-secondary-47.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-47.plan
index 124330a..c823a61 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-47.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-47.plan
@@ -1,10 +1,9 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- DATASOURCE_SCAN (test.testdst)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN (test.testdst)  |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-secondary-48.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-48.plan
index 124330a..c823a61 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-48.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-48.plan
@@ -1,10 +1,9 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- DATASOURCE_SCAN (test.testdst)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN (test.testdst)  |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-secondary-51.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-51.plan
index 124330a..c823a61 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-51.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-51.plan
@@ -1,10 +1,9 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- DATASOURCE_SCAN (test.testdst)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN (test.testdst)  |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-secondary-52.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-52.plan
index 124330a..c823a61 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-52.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-52.plan
@@ -1,10 +1,9 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- DATASOURCE_SCAN (test.testdst)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN (test.testdst)  |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-secondary-53.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-53.plan
index 124330a..c823a61 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-53.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-53.plan
@@ -1,10 +1,9 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- DATASOURCE_SCAN (test.testdst)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN (test.testdst)  |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-secondary-58.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-58.plan
index 124330a..c823a61 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-58.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-58.plan
@@ -1,10 +1,9 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- DATASOURCE_SCAN (test.testdst)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN (test.testdst)  |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-secondary-59.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-59.plan
index 124330a..c823a61 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-59.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-59.plan
@@ -1,10 +1,9 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- DATASOURCE_SCAN (test.testdst)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN (test.testdst)  |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-secondary-61.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-61.plan
index 124330a..c823a61 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-61.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-61.plan
@@ -1,10 +1,9 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- DATASOURCE_SCAN (test.testdst)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN (test.testdst)  |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-secondary-68.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-68.plan
index 4cd09cb..6c20ed9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-68.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-68.plan
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$27(ASC), $$28(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$27(ASC), $$28(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$28(ASC), $$29(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$28(ASC), $$29(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-68_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-68_ps.plan
index c7a7d7c..c5b79e8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-68_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-secondary-68_ps.plan
@@ -1,33 +1,30 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$27(ASC), $$28(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$27(ASC), $$28(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$28(ASC), $$29(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$28(ASC), $$29(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |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 8d3aa97..6dc1292 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
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.MyData)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.MyData)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |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_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01-disable-idxonly_ps.plan
index 3ebae2c..7282a9b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01-disable-idxonly_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01-disable-idxonly_ps.plan
@@ -1,33 +1,30 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$20(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$21(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (test.MyData)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (test.MyData)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test.MyData)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (test.MyData)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |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 8d3aa97..6dc1292 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
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.MyData)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.MyData)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01_ps.plan
index 3ebae2c..7282a9b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01_ps.plan
@@ -1,33 +1,30 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$20(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$21(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (test.MyData)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (test.MyData)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test.MyData)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (test.MyData)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-10.plan
index 33aec0b..8fac4f2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-10.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-10.plan
@@ -4,12 +4,10 @@
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
         -- AGGREGATE  |PARTITIONED|
           -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (twitter.ds_tweet)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (twitter.ds_tweet)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-ternary-inlj/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-ternary-inlj/query4.plan
index a374bc4..2018224 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-ternary-inlj/query4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-ternary-inlj/query4.plan
@@ -1,32 +1,30 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- HYBRID_HASH_JOIN [$$61][$$74]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH (tpcds.customer_address.customer_address)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$78(ASC)]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH (tpcds.customer_demographics.customer_demographics)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STABLE_SORT [$$76(ASC)]  |PARTITIONED|
-                                          -- HASH_PARTITION_EXCHANGE [$$76]  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- HYBRID_HASH_JOIN [$$61][$$74]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (tpcds.customer_address.customer_address)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$78(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (tpcds.customer_demographics.customer_demographics)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$76(ASC)]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$76]  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
                                             -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (tpcds.customer)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (tpcds.customer)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- UNNEST  |UNPARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- UNNEST  |UNPARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q10.plan
index 453e138..ab5c179 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q10.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q10.plan
@@ -1,59 +1,52 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     -- STREAM_LIMIT  |UNPARTITIONED|
-      -- STREAM_PROJECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$$175(DESC) ]  |PARTITIONED|
-            -- STREAM_LIMIT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [topK: 20] [$$175(DESC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- SORT_GROUP_BY[$$182, $$183, $$184, $$185, $$186]  |PARTITIONED|
-                            {
-                              -- AGGREGATE  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                            }
-                      -- HASH_PARTITION_EXCHANGE [$$182, $$183, $$184, $$185, $$186]  |PARTITIONED|
-                        -- SORT_GROUP_BY[$$166, $$161, $$162, $$163, $$164]  |PARTITIONED|
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- HYBRID_HASH_JOIN [$$177][$$176]  |PARTITIONED|
-                                  -- HASH_PARTITION_EXCHANGE [$$177]  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- HYBRID_HASH_JOIN [$$166, $$171, $$173][$$170, $$172, $$174]  |PARTITIONED|
-                                          -- HASH_PARTITION_EXCHANGE [$$166, $$171, $$173]  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$175(DESC) ]  |PARTITIONED|
+          -- STREAM_LIMIT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [topK: 20] [$$175(DESC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- SORT_GROUP_BY[$$184, $$185, $$186, $$187, $$188]  |PARTITIONED|
+                          {
+                            -- AGGREGATE  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                          }
+                    -- HASH_PARTITION_EXCHANGE [$$184, $$185, $$186, $$187, $$188]  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$166, $$161, $$162, $$163, $$164]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$177][$$176]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- UNNEST  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$166, $$171, $$173][$$170, $$172, $$174]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$166, $$171, $$173]  |PARTITIONED|
                                               -- ASSIGN  |PARTITIONED|
                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                     -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                          -- HASH_PARTITION_EXCHANGE [$$170, $$172, $$174]  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- UNNEST  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                  -- HASH_PARTITION_EXCHANGE [$$176]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$170, $$172, $$174]  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q11.plan
index ed6d2b0..52342a6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q11.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q11.plan
@@ -1,62 +1,100 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$188(DESC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$188(DESC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- NESTED_LOOP  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- SORT_GROUP_BY[$$214]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- HASH_PARTITION_EXCHANGE [$$214]  |PARTITIONED|
-                          -- SORT_GROUP_BY[$$178]  |PARTITIONED|
-                                  {
-                                    -- AGGREGATE  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                  }
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$186][$$185]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- HYBRID_HASH_JOIN [$$198][$$197]  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- REPLICATE  |PARTITIONED|
-                                                      -- HASH_PARTITION_EXCHANGE [$$203]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$188(DESC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$188(DESC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- SORT_GROUP_BY[$$214]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- HASH_PARTITION_EXCHANGE [$$214]  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$178]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$186][$$185]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$198][$$197]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- REPLICATE  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$203]  |PARTITIONED|
+                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
                                                         -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- STREAM_SELECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (test.nation)  |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 [$$202]  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- REPLICATE  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$202]  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |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 (test.stock)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- ASSIGN  |UNPARTITIONED|
+                      -- AGGREGATE  |UNPARTITIONED|
+                        -- ASSIGN  |UNPARTITIONED|
+                          -- AGGREGATE  |UNPARTITIONED|
+                            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                              -- AGGREGATE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$201][$$200]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$203][$$202]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- REPLICATE  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$203]  |PARTITIONED|
+                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- REPLICATE  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$202]  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- ASSIGN  |PARTITIONED|
                                           -- ASSIGN  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -67,53 +105,3 @@
                                                       -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |UNPARTITIONED|
-                        -- ASSIGN  |UNPARTITIONED|
-                          -- AGGREGATE  |UNPARTITIONED|
-                            -- STREAM_PROJECT  |UNPARTITIONED|
-                              -- ASSIGN  |UNPARTITIONED|
-                                -- AGGREGATE  |UNPARTITIONED|
-                                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                    -- AGGREGATE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- HYBRID_HASH_JOIN [$$201][$$200]  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- HYBRID_HASH_JOIN [$$203][$$202]  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- REPLICATE  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$203]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- STREAM_SELECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- REPLICATE  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$202]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |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 (test.stock)  |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 715a293..9a2c5f2 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
@@ -1,65 +1,59 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$139(DESC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$139(DESC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- PRE_CLUSTERED_GROUP_BY[$$129, $$130, $$131]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
-                            -- MICRO_STABLE_SORT [$$150(ASC)]  |LOCAL|
-                              -- ASSIGN  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |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[$$133, $$134]  |PARTITIONED|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- STREAM_SELECT  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- 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|
-                                              -- HYBRID_HASH_JOIN [$$138][$$137]  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- NESTED_LOOP  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$139(DESC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$139(DESC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- PRE_CLUSTERED_GROUP_BY[$$129, $$130, $$131]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                          -- MICRO_STABLE_SORT [$$150(ASC)]  |LOCAL|
+                            -- ASSIGN  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$129(ASC), $$130(ASC), $$131(ASC)]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$129, $$130, $$131]  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_CLUSTERED_GROUP_BY[$$133, $$134]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- 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|
+                                        -- HYBRID_HASH_JOIN [$$138][$$137]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- NESTED_LOOP  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
                                                           -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- STREAM_SELECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                  -- STREAM_SELECT  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q18.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q18.plan
new file mode 100644
index 0000000..27ee1f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q18.plan
@@ -0,0 +1,44 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$193(DESC), $$o_entry_d(ASC) ]  |PARTITIONED|
+            -- STREAM_LIMIT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [topK: 100] [$$193(DESC), $$o_entry_d(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- SORT_GROUP_BY[$$201, $$202, $$203, $$204, $$205, $$206, $$207]  |PARTITIONED|
+                                  {
+                                    -- AGGREGATE  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                  }
+                            -- HASH_PARTITION_EXCHANGE [$$201, $$202, $$203, $$204, $$205, $$206, $$207]  |PARTITIONED|
+                              -- SORT_GROUP_BY[$$175, $$183, $$184, $$185, $$179, $$180, $$181]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- UNNEST  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$188, $$183, $$184][$$185, $$189, $$190]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$188, $$183, $$184]  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$185, $$189, $$190]  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q2.plan
index c1d568c..e36e0c8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q2.plan
@@ -1,162 +1,147 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     -- STREAM_LIMIT  |UNPARTITIONED|
-      -- STREAM_PROJECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$$274(ASC), $$275(ASC), $$240(ASC) ]  |PARTITIONED|
-            -- STREAM_LIMIT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [topK: 100] [$$274(ASC), $$275(ASC), $$240(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- HYBRID_HASH_JOIN [$$252][$$253]  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$252]  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- HYBRID_HASH_JOIN [$$259][$$260]  |PARTITIONED|
-                                  -- HASH_PARTITION_EXCHANGE [$$259]  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- HYBRID_HASH_JOIN [$$265][$$242]  |PARTITIONED|
-                                          -- HASH_PARTITION_EXCHANGE [$$265]  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$280(ASC), $$281(ASC), $$247(ASC) ]  |PARTITIONED|
+          -- STREAM_LIMIT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [topK: 100] [$$280(ASC), $$281(ASC), $$247(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$260][$$261]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$260]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$267][$$268]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$267]  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- HYBRID_HASH_JOIN [$$273][$$249]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$273]  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
                                             -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- HYBRID_HASH_JOIN [$$240, $$254][$$241, $$263]  |PARTITIONED|
-                                                      -- HASH_PARTITION_EXCHANGE [$$240, $$254]  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- HYBRID_HASH_JOIN [$$247, $$262][$$248, $$271]  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$247, $$262]  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- HYBRID_HASH_JOIN [$$s_i_id][$$247]  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- HYBRID_HASH_JOIN [$$s_i_id][$$240]  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- SORT_GROUP_BY[$$285]  |PARTITIONED|
+                                                            -- SORT_GROUP_BY[$$291]  |PARTITIONED|
+                                                                    {
+                                                                      -- AGGREGATE  |LOCAL|
+                                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                    }
+                                                              -- HASH_PARTITION_EXCHANGE [$$291]  |PARTITIONED|
+                                                                -- SORT_GROUP_BY[$$245]  |PARTITIONED|
                                                                         {
                                                                           -- AGGREGATE  |LOCAL|
                                                                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                                         }
-                                                                  -- HASH_PARTITION_EXCHANGE [$$285]  |PARTITIONED|
-                                                                    -- SORT_GROUP_BY[$$239]  |PARTITIONED|
-                                                                            {
-                                                                              -- AGGREGATE  |LOCAL|
-                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                            }
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- STREAM_PROJECT  |PARTITIONED|
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- HYBRID_HASH_JOIN [$$263][$$264]  |PARTITIONED|
                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- HYBRID_HASH_JOIN [$$255][$$256]  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    -- HYBRID_HASH_JOIN [$$257][$$269]  |PARTITIONED|
-                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            -- HYBRID_HASH_JOIN [$$261][$$270]  |PARTITIONED|
-                                                                                              -- HASH_PARTITION_EXCHANGE [$$261]  |PARTITIONED|
-                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                  -- ASSIGN  |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 (test.stock)  |PARTITIONED|
-                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                              -- HASH_PARTITION_EXCHANGE [$$270]  |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 (test.supplier)  |PARTITIONED|
-                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                      -- BROADCAST_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 (test.nation)  |PARTITIONED|
-                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                -- HYBRID_HASH_JOIN [$$265][$$276]  |PARTITIONED|
                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                     -- STREAM_PROJECT  |PARTITIONED|
-                                                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                                                        -- ASSIGN  |PARTITIONED|
-                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                              -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- HYBRID_HASH_JOIN [$$269][$$277]  |PARTITIONED|
+                                                                                          -- HASH_PARTITION_EXCHANGE [$$269]  |PARTITIONED|
+                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                              -- ASSIGN  |PARTITIONED|
                                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                              -- HASH_PARTITION_EXCHANGE [$$240]  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- STREAM_SELECT  |PARTITIONED|
-                                                                    -- ASSIGN  |PARTITIONED|
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                      -- HASH_PARTITION_EXCHANGE [$$241, $$263]  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                          -- HASH_PARTITION_EXCHANGE [$$277]  |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 (test.supplier)  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                  -- BROADCAST_EXCHANGE  |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 (test.nation)  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                            -- REPLICATE  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- HASH_PARTITION_EXCHANGE [$$247]  |PARTITIONED|
+                                                            -- STREAM_SELECT  |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 (test.stock)  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                          -- HASH_PARTITION_EXCHANGE [$$242]  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- REPLICATE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$248, $$271]  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- REPLICATE  |PARTITIONED|
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                  -- HASH_PARTITION_EXCHANGE [$$260]  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$249]  |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 (test.nation)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$253]  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- REPLICATE  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- STREAM_SELECT  |PARTITIONED|
-                                          -- ASSIGN  |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 (test.supplier)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$268]  |PARTITIONED|
+                                  -- ASSIGN  |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 (test.region)  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$261]  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q21.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q21.plan
index 689bec3..c9b6770 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q21.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q21.plan
@@ -3,114 +3,100 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- SORT_MERGE_EXCHANGE [$$su_name(ASC) ]  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_LIMIT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- SORT_GROUP_BY[$$399]  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- HASH_PARTITION_EXCHANGE [$$399]  |PARTITIONED|
-                      -- SORT_GROUP_BY[$$su_name]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_LIMIT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- SORT_GROUP_BY[$$399]  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- HASH_PARTITION_EXCHANGE [$$399]  |PARTITIONED|
+                    -- SORT_GROUP_BY[$$su_name]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- SORT_GROUP_BY[$$390, $$391, $$392, $$393, $$394, $$395, $$396, $$397]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- SORT_GROUP_BY[$$390, $$391, $$392, $$393, $$394, $$395, $$396, $$397]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- HASH_PARTITION_EXCHANGE [$$390, $$391, $$392, $$393, $$394, $$395, $$396, $$397]  |PARTITIONED|
+                                  -- SORT_GROUP_BY[$$352, $$378, $$377, $$354, $$355, $$351, $$350, $$379]  |PARTITIONED|
                                           {
                                             -- AGGREGATE  |LOCAL|
                                               -- NESTED_TUPLE_SOURCE  |LOCAL|
                                           }
-                                    -- HASH_PARTITION_EXCHANGE [$$390, $$391, $$392, $$393, $$394, $$395, $$396, $$397]  |PARTITIONED|
-                                      -- SORT_GROUP_BY[$$352, $$378, $$377, $$354, $$355, $$351, $$350, $$379]  |PARTITIONED|
-                                              {
-                                                -- AGGREGATE  |LOCAL|
-                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                              }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$365][$$354]  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$365][$$354]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- HYBRID_HASH_JOIN [$$366][$$355]  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$366]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- HYBRID_HASH_JOIN [$$352, $$362][$$351, $$350]  |PARTITIONED|
-                                                                -- HASH_PARTITION_EXCHANGE [$$352, $$362]  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- NESTED_LOOP  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- STREAM_SELECT  |PARTITIONED|
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                    -- UNNEST  |PARTITIONED|
-                                                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                          -- ASSIGN  |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 (test.orders)  |PARTITIONED|
-                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                  -- UNNEST  |PARTITIONED|
-                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                      -- STREAM_SELECT  |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 (test.orders)  |PARTITIONED|
-                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                -- HASH_PARTITION_EXCHANGE [$$351, $$350]  |PARTITIONED|
-                                                                  -- ASSIGN  |PARTITIONED|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                      -- ASSIGN  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$355]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
+                                                  -- HYBRID_HASH_JOIN [$$366][$$355]  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$366]  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- HYBRID_HASH_JOIN [$$352, $$362][$$351, $$350]  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$352, $$362]  |PARTITIONED|
                                                               -- STREAM_PROJECT  |PARTITIONED|
                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                                                  -- NESTED_LOOP  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- UNNEST  |PARTITIONED|
+                                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                                              -- ASSIGN  |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 (test.orders)  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- UNNEST  |PARTITIONED|
+                                                                            -- STREAM_SELECT  |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 (test.orders)  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$351, $$350]  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$355]  |PARTITIONED|
                                                       -- ASSIGN  |PARTITIONED|
                                                         -- STREAM_PROJECT  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q5.plan
index 8de5d0d..8e1e3d7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q5.plan
@@ -1,98 +1,86 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$#2(DESC) ]  |PARTITIONED|
-          -- STABLE_SORT [$#2(DESC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- SORT_GROUP_BY[$$292]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$#2(DESC) ]  |PARTITIONED|
+        -- STABLE_SORT [$#2(DESC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- SORT_GROUP_BY[$$292]  |PARTITIONED|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- HASH_PARTITION_EXCHANGE [$$292]  |PARTITIONED|
+                    -- SORT_GROUP_BY[$$275]  |PARTITIONED|
                             {
                               -- AGGREGATE  |LOCAL|
                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                             }
-                      -- HASH_PARTITION_EXCHANGE [$$292]  |PARTITIONED|
-                        -- SORT_GROUP_BY[$$275]  |PARTITIONED|
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$274, $$290][$$266, $$269]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- HYBRID_HASH_JOIN [$$274, $$290][$$266, $$269]  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- HYBRID_HASH_JOIN [$$256][$$257]  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- HYBRID_HASH_JOIN [$$258][$$274]  |PARTITIONED|
-                                                      -- HASH_PARTITION_EXCHANGE [$$258]  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- HYBRID_HASH_JOIN [$$260, $$254, $$263][$$276, $$277, $$278]  |PARTITIONED|
-                                                              -- HASH_PARTITION_EXCHANGE [$$260, $$254, $$263]  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- HYBRID_HASH_JOIN [$$245, $$246][$$254, $$281]  |PARTITIONED|
-                                                                      -- HASH_PARTITION_EXCHANGE [$$245, $$246]  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
-                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                      -- HASH_PARTITION_EXCHANGE [$$254, $$281]  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                              -- UNNEST  |PARTITIONED|
-                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                  -- STREAM_SELECT  |PARTITIONED|
-                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                              -- HASH_PARTITION_EXCHANGE [$$276, $$277, $$278]  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- HYBRID_HASH_JOIN [$$256][$$257]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- HYBRID_HASH_JOIN [$$258][$$274]  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$258]  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- HYBRID_HASH_JOIN [$$260, $$254, $$263][$$276, $$277, $$278]  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$260, $$254, $$263]  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- HYBRID_HASH_JOIN [$$245, $$246][$$254, $$281]  |PARTITIONED|
+                                                                -- HASH_PARTITION_EXCHANGE [$$245, $$246]  |PARTITIONED|
                                                                   -- ASSIGN  |PARTITIONED|
                                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
+                                                                        -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                      -- HASH_PARTITION_EXCHANGE [$$274]  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- HASH_PARTITION_EXCHANGE [$$254, $$281]  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- UNNEST  |PARTITIONED|
+                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$276, $$277, $$278]  |PARTITIONED|
                                                           -- ASSIGN  |PARTITIONED|
                                                             -- STREAM_PROJECT  |PARTITIONED|
                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- STREAM_SELECT  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- DATASOURCE_SCAN (test.region)  |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|
-                                            -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$274]  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q7.plan
index 0eaee70..0334a38 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q7.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q7.plan
@@ -1,104 +1,90 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$su_nationkey(ASC), $#1(ASC), $#2(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$328, $$329, $$330]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$328, $$329, $$330]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$277, $$273, $$274]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- HYBRID_HASH_JOIN [$$288][$$304]  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- HYBRID_HASH_JOIN [$$277][$$303]  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$326][$$300]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$326]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$su_nationkey(ASC), $#1(ASC), $#2(ASC) ]  |PARTITIONED|
+        -- SORT_GROUP_BY[$$328, $$329, $$330]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$328, $$329, $$330]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$277, $$273, $$274]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$288][$$304]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$277][$$303]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$326][$$300]  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$326]  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- HYBRID_HASH_JOIN [$$291, $$293, $$295][$$305, $$306, $$307]  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$291, $$293, $$295]  |PARTITIONED|
                                                       -- STREAM_PROJECT  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- HYBRID_HASH_JOIN [$$291, $$293, $$295][$$305, $$306, $$307]  |PARTITIONED|
-                                                            -- HASH_PARTITION_EXCHANGE [$$291, $$293, $$295]  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- HYBRID_HASH_JOIN [$$275, $$276][$$310, $$311]  |PARTITIONED|
-                                                                    -- HASH_PARTITION_EXCHANGE [$$275, $$276]  |PARTITIONED|
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        -- ASSIGN  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                    -- HASH_PARTITION_EXCHANGE [$$310, $$311]  |PARTITIONED|
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        -- STREAM_SELECT  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                -- UNNEST  |PARTITIONED|
-                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                          -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                            -- HASH_PARTITION_EXCHANGE [$$305, $$306, $$307]  |PARTITIONED|
+                                                          -- HYBRID_HASH_JOIN [$$275, $$276][$$310, $$311]  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$275, $$276]  |PARTITIONED|
                                                               -- ASSIGN  |PARTITIONED|
                                                                 -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ASSIGN  |PARTITIONED|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$300]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$310, $$311]  |PARTITIONED|
+                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- UNNEST  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$305, $$306, $$307]  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- REPLICATE  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$300]  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
                                               -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- REPLICATE  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- ASSIGN  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- REPLICATE  |PARTITIONED|
-                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q8.plan
index 4223541..d3a0c1a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q8.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q8.plan
@@ -1,128 +1,114 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$#1(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$351]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$351]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$294]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$#1(ASC) ]  |PARTITIONED|
+        -- SORT_GROUP_BY[$$351]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$351]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$294]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$325][$$326]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- HYBRID_HASH_JOIN [$$325][$$326]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$348][$$341]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$348][$$341]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$308][$$309]  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- HYBRID_HASH_JOIN [$$311][$$331]  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- HYBRID_HASH_JOIN [$$306][$$324]  |PARTITIONED|
-                                                                -- HASH_PARTITION_EXCHANGE [$$306]  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- HYBRID_HASH_JOIN [$$313, $$315, $$317][$$333, $$334, $$332]  |PARTITIONED|
-                                                                        -- HASH_PARTITION_EXCHANGE [$$313, $$315, $$317]  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- HYBRID_HASH_JOIN [$$296, $$295][$$306, $$336]  |PARTITIONED|
-                                                                                -- HASH_PARTITION_EXCHANGE [$$296, $$295]  |PARTITIONED|
-                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                          -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
-                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                -- HASH_PARTITION_EXCHANGE [$$306, $$336]  |PARTITIONED|
-                                                                                  -- STREAM_SELECT  |PARTITIONED|
-                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                          -- UNNEST  |PARTITIONED|
-                                                                                            -- STREAM_SELECT  |PARTITIONED|
-                                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                      -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                        -- HASH_PARTITION_EXCHANGE [$$333, $$334, $$332]  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                -- HASH_PARTITION_EXCHANGE [$$324]  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                                      -- ASSIGN  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                        -- BROADCAST_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 (test.nation)  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- DATASOURCE_SCAN (test.region)  |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|
-                                              -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- REPLICATE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$308][$$309]  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- STREAM_PROJECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                                -- HYBRID_HASH_JOIN [$$311][$$331]  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- HYBRID_HASH_JOIN [$$306][$$324]  |PARTITIONED|
+                                                          -- HASH_PARTITION_EXCHANGE [$$306]  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- HYBRID_HASH_JOIN [$$313, $$315, $$317][$$333, $$334, $$332]  |PARTITIONED|
+                                                                  -- HASH_PARTITION_EXCHANGE [$$313, $$315, $$317]  |PARTITIONED|
+                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- HYBRID_HASH_JOIN [$$296, $$295][$$306, $$336]  |PARTITIONED|
+                                                                          -- HASH_PARTITION_EXCHANGE [$$296, $$295]  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                          -- HASH_PARTITION_EXCHANGE [$$306, $$336]  |PARTITIONED|
+                                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- UNNEST  |PARTITIONED|
+                                                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                  -- HASH_PARTITION_EXCHANGE [$$333, $$334, $$332]  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- HASH_PARTITION_EXCHANGE [$$324]  |PARTITIONED|
+                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |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 (test.nation)  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |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 (test.nation)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q9.plan
index 0fb4fff..d59c9cf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q9.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q9.plan
@@ -1,81 +1,71 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$n_name(ASC), $#1(DESC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$231, $$232]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$231, $$232]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$225, $$198]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$n_name(ASC), $#1(DESC) ]  |PARTITIONED|
+        -- SORT_GROUP_BY[$$231, $$232]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$231, $$232]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$225, $$198]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$213][$$214]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- HYBRID_HASH_JOIN [$$213][$$214]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$229][$$224]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$229][$$224]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$206][$$207]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$206]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- HYBRID_HASH_JOIN [$$200, $$199][$$206, $$217]  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$200, $$199]  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$206][$$207]  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$206]  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- HYBRID_HASH_JOIN [$$200, $$199][$$206, $$217]  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$200, $$199]  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$206, $$217]  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- UNNEST  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
                                                           -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$206, $$217]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- UNNEST  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ASSIGN  |PARTITIONED|
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$207]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- DATASOURCE_SCAN (test.item)  |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|
-                                              -- DATASOURCE_SCAN (test.supplier)  |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|
-                                      -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$207]  |PARTITIONED|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/column-pushdown/meta.001.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/column-pushdown/meta.001.plan
index d90d07b..1530eeb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/column-pushdown/meta.001.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/column-pushdown/meta.001.plan
@@ -1,62 +1,100 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$194(DESC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$194(DESC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- NESTED_LOOP  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- SORT_GROUP_BY[$$223]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- HASH_PARTITION_EXCHANGE [$$223]  |PARTITIONED|
-                          -- SORT_GROUP_BY[$$178]  |PARTITIONED|
-                                  {
-                                    -- AGGREGATE  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                  }
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$192][$$191]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- HYBRID_HASH_JOIN [$$207][$$206]  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- REPLICATE  |PARTITIONED|
-                                                      -- HASH_PARTITION_EXCHANGE [$$212]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$194(DESC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$194(DESC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- SORT_GROUP_BY[$$223]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- HASH_PARTITION_EXCHANGE [$$223]  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$178]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$192][$$191]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$207][$$206]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- REPLICATE  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$212]  |PARTITIONED|
+                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
                                                         -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- STREAM_SELECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (test.nation)  |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 [$$211]  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- REPLICATE  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$211]  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |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 (test.stock)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- ASSIGN  |UNPARTITIONED|
+                      -- AGGREGATE  |UNPARTITIONED|
+                        -- ASSIGN  |UNPARTITIONED|
+                          -- AGGREGATE  |UNPARTITIONED|
+                            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                              -- AGGREGATE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$210][$$209]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$212][$$211]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- REPLICATE  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$212]  |PARTITIONED|
+                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- REPLICATE  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$211]  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- ASSIGN  |PARTITIONED|
                                           -- ASSIGN  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -67,53 +105,3 @@
                                                       -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |UNPARTITIONED|
-                        -- ASSIGN  |UNPARTITIONED|
-                          -- AGGREGATE  |UNPARTITIONED|
-                            -- STREAM_PROJECT  |UNPARTITIONED|
-                              -- ASSIGN  |UNPARTITIONED|
-                                -- AGGREGATE  |UNPARTITIONED|
-                                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                    -- AGGREGATE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- HYBRID_HASH_JOIN [$$210][$$209]  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- HYBRID_HASH_JOIN [$$212][$$211]  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- REPLICATE  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$212]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- STREAM_SELECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- REPLICATE  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$211]  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |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 (test.stock)  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/extract-common-operators/extract-common-operators.01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/extract-common-operators/extract-common-operators.01.plan
index ae07a55..e0c2670 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/extract-common-operators/extract-common-operators.01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/extract-common-operators/extract-common-operators.01.plan
@@ -1,97 +1,92 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$405(ASC), $$406(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$405(ASC), $$406(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- HYBRID_HASH_JOIN [$$414][$$410]  |PARTITIONED|
-                    -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$405(ASC), $$406(ASC)] HASH:[$$414]  |PARTITIONED|
-                      -- STABLE_SORT [$$405(ASC), $$406(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- HYBRID_HASH_JOIN [$$414][$$408]  |PARTITIONED|
-                                -- HASH_PARTITION_EXCHANGE [$$414]  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- REPLICATE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN (Metadata.Synonym)  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- HASH_PARTITION_EXCHANGE [$$408]  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- HYBRID_HASH_JOIN [$$408, $$407][$$ds_name, $$dv_name]  |PARTITIONED|
-                                        -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                                          -- REPLICATE  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- DATASOURCE_SCAN (Metadata.Dataset)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                          -- NESTED_LOOP  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                              -- STREAM_PROJECT  |UNPARTITIONED|
-                                                -- ASSIGN  |UNPARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                    -- REPLICATE  |UNPARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                        -- UNNEST  |UNPARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                              -- UNNEST  |UNPARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$410]  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- HYBRID_HASH_JOIN [$$428, $$ds_name, $$dv_name][$$412, $$443, $$411]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$410][$$syn_name]  |PARTITIONED|
-                                    -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- REPLICATE  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- DATASOURCE_SCAN (Metadata.Synonym)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                      -- NESTED_LOOP  |PARTITIONED|
-                                        -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                                          -- NESTED_LOOP  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                              -- UNNEST  |UNPARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                              -- REPLICATE  |UNPARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                  -- UNNEST  |UNPARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                          -- UNNEST  |UNPARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$405(ASC), $$406(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$405(ASC), $$406(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- HYBRID_HASH_JOIN [$$414][$$410]  |PARTITIONED|
+                  -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$405(ASC), $$406(ASC)] HASH:[$$414]  |PARTITIONED|
+                    -- STABLE_SORT [$$405(ASC), $$406(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$414][$$408]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$414]  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- REPLICATE  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (Metadata.Synonym)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (Metadata.Dataset)  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$408]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$408, $$407][$$ds_name, $$dv_name]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$408, $$407]  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (Metadata.Dataset)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$ds_name, $$dv_name]  |PARTITIONED|
+                                        -- NESTED_LOOP  |UNPARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                            -- UNNEST  |UNPARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                            -- REPLICATE  |UNPARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$410]  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$428][$$412]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$428]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$410][$$syn_name]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$410]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- REPLICATE  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (Metadata.Synonym)  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$syn_name]  |PARTITIONED|
+                                    -- UNNEST  |UNPARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$412]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$412, $$411][$$ds_name, $$dv_name]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$408, $$407]  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (Metadata.Dataset)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$ds_name, $$dv_name]  |PARTITIONED|
+                                    -- NESTED_LOOP  |UNPARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                        -- ASSIGN  |UNPARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                            -- REPLICATE  |UNPARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                        -- UNNEST  |UNPARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/filter/inverted-btree-search-return-optional-field.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/filter/inverted-btree-search-return-optional-field.plan
index 6993f25..a76d642 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/filter/inverted-btree-search-return-optional-field.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/filter/inverted-btree-search-return-optional-field.plan
@@ -1,26 +1,27 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     -- STREAM_LIMIT  |UNPARTITIONED|
-      -- STREAM_PROJECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$$39(DESC) ]  |PARTITIONED|
-            -- STREAM_LIMIT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [topK: 2147483647] [$$39(DESC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- HYBRID_HASH_JOIN [$$26][$$41]  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- STREAM_SELECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (twitter.ds_tweet)  |PARTITIONED|
-                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- UNNEST  |UNPARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$40(DESC) ]  |PARTITIONED|
+          -- STREAM_LIMIT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [topK: 2147483647] [$$40(DESC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$27][$$42]  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (twitter.ds_tweet.ds_tweet)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$53(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- SINGLE_PARTITION_INVERTED_INDEX_SEARCH (twitter.ds_tweet.text_idx)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- UNNEST  |UNPARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/filter_on_meta_with_idx_2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/filter_on_meta_with_idx_2.plan
index 05da710..cccf7a1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/filter_on_meta_with_idx_2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/filter_on_meta_with_idx_2.plan
@@ -1,25 +1,22 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$37][$$38]  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.KVStore)  |PARTITIONED|
-                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.OfficerLocations)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- HYBRID_HASH_JOIN [$$37][$$38]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.KVStore)  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (test.OfficerLocations)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/group-by/listify-3.1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/group-by/listify-3.1.plan
index ec40853..d28f82a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/group-by/listify-3.1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/group-by/listify-3.1.plan
@@ -1,43 +1,36 @@
 -- DISTRIBUTE_RESULT  |LOCAL|
   -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-    -- STREAM_PROJECT  |LOCAL|
-      -- ASSIGN  |LOCAL|
+    -- ASSIGN  |LOCAL|
+      -- AGGREGATE  |LOCAL|
         -- AGGREGATE  |LOCAL|
-          -- AGGREGATE  |LOCAL|
-            -- STREAM_PROJECT  |LOCAL|
-              -- ASSIGN  |LOCAL|
+          -- ASSIGN  |LOCAL|
+            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+              -- NESTED_LOOP  |LOCAL|
                 -- ONE_TO_ONE_EXCHANGE  |LOCAL|
                   -- NESTED_LOOP  |LOCAL|
                     -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
                       -- UNNEST  |UNPARTITIONED|
                         -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                      -- NESTED_LOOP  |LOCAL|
+                      -- ASSIGN  |LOCAL|
                         -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                          -- STREAM_PROJECT  |LOCAL|
-                            -- ASSIGN  |LOCAL|
-                              -- STREAM_PROJECT  |LOCAL|
+                          -- REPLICATE  |LOCAL|
+                            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                              -- AGGREGATE  |LOCAL|
                                 -- ASSIGN  |LOCAL|
-                                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                                    -- REPLICATE  |LOCAL|
-                                      -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                                        -- AGGREGATE  |LOCAL|
-                                          -- STREAM_PROJECT  |LOCAL|
-                                            -- ASSIGN  |LOCAL|
-                                              -- AGGREGATE  |LOCAL|
-                                                -- AGGREGATE  |LOCAL|
-                                                  -- UNNEST  |UNPARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                          -- STREAM_PROJECT  |LOCAL|
-                            -- ASSIGN  |LOCAL|
-                              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                                -- REPLICATE  |LOCAL|
-                                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                                  -- AGGREGATE  |LOCAL|
                                     -- AGGREGATE  |LOCAL|
-                                      -- STREAM_PROJECT  |LOCAL|
-                                        -- ASSIGN  |LOCAL|
-                                          -- AGGREGATE  |LOCAL|
-                                            -- AGGREGATE  |LOCAL|
-                                              -- UNNEST  |UNPARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                      -- UNNEST  |UNPARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- ASSIGN  |LOCAL|
+                    -- ASSIGN  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                        -- REPLICATE  |LOCAL|
+                          -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                            -- AGGREGATE  |LOCAL|
+                              -- ASSIGN  |LOCAL|
+                                -- AGGREGATE  |LOCAL|
+                                  -- AGGREGATE  |LOCAL|
+                                    -- UNNEST  |UNPARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/hash-join-with-redundant-variable/hash-join-with-redundant-variable.1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/hash-join-with-redundant-variable/hash-join-with-redundant-variable.1.plan
index ce3065b..310ef7c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/hash-join-with-redundant-variable/hash-join-with-redundant-variable.1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/hash-join-with-redundant-variable/hash-join-with-redundant-variable.1.plan
@@ -22,53 +22,46 @@
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- HYBRID_HASH_JOIN [$$137][$$136]  |PARTITIONED|
                                               -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- STREAM_SELECT  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$136]  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                -- STREAM_SELECT  |PARTITIONED|
                                                   -- ASSIGN  |PARTITIONED|
                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$136]  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (test.nation)  |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|
-                                                -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                               -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                       -- BROADCAST_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.lineitem)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.lineitem)  |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|
-                        -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/hash-join-with-redundant-variable/hash-join-with-redundant-variable.2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/hash-join-with-redundant-variable/hash-join-with-redundant-variable.2.plan
index cdd3368..33b0e85 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/hash-join-with-redundant-variable/hash-join-with-redundant-variable.2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/hash-join-with-redundant-variable/hash-join-with-redundant-variable.2.plan
@@ -10,26 +10,23 @@
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- HYBRID_HASH_JOIN [$$69][$$68]  |PARTITIONED|
                       -- HASH_PARTITION_EXCHANGE [$$69]  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                       -- HASH_PARTITION_EXCHANGE [$$68]  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.orders)  |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|
-                        -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/hash-join-with-redundant-variable/hash-join-with-redundant-variable.3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/hash-join-with-redundant-variable/hash-join-with-redundant-variable.3.plan
index 6a2c579..e3bc303 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/hash-join-with-redundant-variable/hash-join-with-redundant-variable.3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/hash-join-with-redundant-variable/hash-join-with-redundant-variable.3.plan
@@ -11,27 +11,24 @@
                     -- HYBRID_HASH_JOIN [$$71, $$79][$$70, $$78]  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
               -- HASH_PARTITION_EXCHANGE [$$72]  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.1.plan
index f0f6a03..aefe73e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.1.plan
@@ -1,11 +1,10 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.2.plan
index f0f6a03..aefe73e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.2.plan
@@ -1,11 +1,10 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.3.plan
index f0f6a03..aefe73e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.3.plan
@@ -1,11 +1,10 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.4.plan
index f0f6a03..aefe73e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.4.plan
@@ -1,11 +1,10 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.8.plan
index f0f6a03..aefe73e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.8.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.8.plan
@@ -1,11 +1,10 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- ASSIGN  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.9.plan
index a7f92e3..e16c7f5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.9.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.9.plan
@@ -1,25 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$88][$$89]  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$88]  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- ASSIGN  |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 (Test.Users)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$89]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- HYBRID_HASH_JOIN [$$88][$$89]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$88]  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
                 -- STREAM_SELECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -31,3 +17,15 @@
                                 -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$89]  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ASSIGN  |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 (Test.Users)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/insert-and-scan-dataset.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/insert-and-scan-dataset.plan
index f556169..0fba5fc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/insert-and-scan-dataset.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/insert-and-scan-dataset.plan
@@ -6,10 +6,9 @@
           -- MATERIALIZE  |PARTITIONED|
             -- HASH_PARTITION_EXCHANGE [$$17]  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.myData)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |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/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan
index 6d1d661..b0c4d3b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan
@@ -1,15 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |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_cbo/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan
index 6d1d661..b0c4d3b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan
@@ -1,15 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |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_cbo/inverted-index-complex/ngram-edit-distance-check-substring.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-complex/ngram-edit-distance-check-substring.plan
index 2b26d45..3321d55 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-complex/ngram-edit-distance-check-substring.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-complex/ngram-edit-distance-check-substring.plan
@@ -1,16 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-complex/ngram-jaccard-check-multi-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-complex/ngram-jaccard-check-multi-let.plan
index 1d0f0a4..fb9a209 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-complex/ngram-jaccard-check-multi-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-complex/ngram-jaccard-check-multi-let.plan
@@ -1,17 +1,15 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
           -- STREAM_PROJECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                  -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
+                      -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-complex/word-jaccard-check-multi-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-complex/word-jaccard-check-multi-let.plan
index 99e1895..9feb07e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-complex/word-jaccard-check-multi-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-complex/word-jaccard-check-multi-let.plan
@@ -1,17 +1,15 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
           -- STREAM_PROJECT  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                  -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
+                      -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.keyword_index)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.keyword_index)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/joins/inner_right_corr.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/joins/inner_right_corr.plan
index 8327ffc..409f99b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/joins/inner_right_corr.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/joins/inner_right_corr.plan
@@ -1,55 +1,51 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- PRE_CLUSTERED_GROUP_BY[$$76]  |PARTITIONED|
-                    {
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$76]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
                       -- AGGREGATE  |LOCAL|
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |LOCAL|
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                    }
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$76(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- HYBRID_HASH_JOIN [$$76][$$92]  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$76]  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$92]  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+                        -- STREAM_SELECT  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$76(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$76][$$92]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$76]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$92]  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- UNNEST  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
-                                -- UNNEST  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- SUBPLAN  |PARTITIONED|
-                                            {
-                                              -- AGGREGATE  |LOCAL|
-                                                -- STREAM_SELECT  |LOCAL|
-                                                  -- ASSIGN  |LOCAL|
-                                                    -- UNNEST  |LOCAL|
-                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                            }
+                                -- SUBPLAN  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- ASSIGN  |LOCAL|
+                                                -- UNNEST  |LOCAL|
+                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- NESTED_LOOP  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- NESTED_LOOP  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
+                                            -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.t2)  |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|
-                                                    -- DATASOURCE_SCAN (test.t2)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/joins/nlj_partitioning_property_1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/joins/nlj_partitioning_property_1.plan
index 5f3c681..2bb3346 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/joins/nlj_partitioning_property_1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/joins/nlj_partitioning_property_1.plan
@@ -1,29 +1,28 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- HYBRID_HASH_JOIN [$$76][$$78]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- NESTED_LOOP  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (tpch.Supplier)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (tpch.Part)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (tpch.Partsupp)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- HYBRID_HASH_JOIN [$$76][$$78]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- NESTED_LOOP  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (tpch.Supplier)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (tpch.Part)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (tpch.Partsupp)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/joins/nlj_partitioning_property_2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/joins/nlj_partitioning_property_2.plan
index c1554e2..5d9a2c0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/joins/nlj_partitioning_property_2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/joins/nlj_partitioning_property_2.plan
@@ -1,29 +1,28 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- NESTED_LOOP  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- NESTED_LOOP  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (tpch.Supplier)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (tpch.Partsupp)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (tpch.Part)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- NESTED_LOOP  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (tpch.Supplier)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (tpch.Partsupp)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (tpch.Part)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/leftouterjoin/loj-03-no-listify.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/leftouterjoin/loj-03-no-listify.plan
index e57acc8..b39c0dd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/leftouterjoin/loj-03-no-listify.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/leftouterjoin/loj-03-no-listify.plan
@@ -1,36 +1,59 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$taskId(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$taskId(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- HYBRID_HASH_JOIN [$$taskId][$$taskId]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- HYBRID_HASH_JOIN [$$taskId][$$taskId]  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$taskId][$$taskId]  |PARTITIONED|
-                                    -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$taskId(ASC)] HASH:[$$taskId]  |PARTITIONED|
-                                      -- SORT_GROUP_BY[$$295]  |PARTITIONED|
-                                              {
-                                                -- AGGREGATE  |LOCAL|
-                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                              }
-                                        -- HASH_PARTITION_EXCHANGE [$$295]  |PARTITIONED|
-                                          -- SORT_GROUP_BY[$$258]  |PARTITIONED|
-                                                  {
-                                                    -- AGGREGATE  |LOCAL|
-                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                  }
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$taskId(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$taskId(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- HYBRID_HASH_JOIN [$$taskId][$$taskId]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$taskId][$$taskId]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$taskId][$$taskId]  |PARTITIONED|
+                                  -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$taskId(ASC)] HASH:[$$taskId]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$295]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- HASH_PARTITION_EXCHANGE [$$295]  |PARTITIONED|
+                                        -- SORT_GROUP_BY[$$258]  |PARTITIONED|
+                                                {
+                                                  -- AGGREGATE  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                }
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- REPLICATE  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (test.tasks)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- SORT_GROUP_BY[$$297]  |PARTITIONED|
+                                                {
+                                                  -- AGGREGATE  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                }
+                                          -- HASH_PARTITION_EXCHANGE [$$297]  |PARTITIONED|
+                                            -- SORT_GROUP_BY[$$259]  |PARTITIONED|
+                                                    {
+                                                      -- AGGREGATE  |LOCAL|
+                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                    }
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_SELECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                     -- REPLICATE  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -40,81 +63,22 @@
                                                               -- DATASOURCE_SCAN (test.tasks)  |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|
-                                            -- SORT_GROUP_BY[$$297]  |PARTITIONED|
-                                                    {
-                                                      -- AGGREGATE  |LOCAL|
-                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                    }
-                                              -- HASH_PARTITION_EXCHANGE [$$297]  |PARTITIONED|
-                                                -- SORT_GROUP_BY[$$259]  |PARTITIONED|
-                                                        {
-                                                          -- AGGREGATE  |LOCAL|
-                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                        }
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- REPLICATE  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN (test.tasks)  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- SORT_GROUP_BY[$$299]  |PARTITIONED|
-                                            {
-                                              -- AGGREGATE  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                            }
-                                      -- HASH_PARTITION_EXCHANGE [$$299]  |PARTITIONED|
-                                        -- SORT_GROUP_BY[$$260]  |PARTITIONED|
-                                                {
-                                                  -- AGGREGATE  |LOCAL|
-                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                }
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- REPLICATE  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- DATASOURCE_SCAN (test.tasks)  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- SORT_GROUP_BY[$$301]  |PARTITIONED|
-                                    {
-                                      -- AGGREGATE  |LOCAL|
-                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                    }
-                              -- HASH_PARTITION_EXCHANGE [$$301]  |PARTITIONED|
-                                -- SORT_GROUP_BY[$$261]  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$299]  |PARTITIONED|
                                         {
                                           -- AGGREGATE  |LOCAL|
                                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                                         }
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- STREAM_SELECT  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$299]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$260]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
                                           -- ASSIGN  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                               -- REPLICATE  |PARTITIONED|
@@ -125,3 +89,29 @@
                                                         -- DATASOURCE_SCAN (test.tasks)  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$301]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- HASH_PARTITION_EXCHANGE [$$301]  |PARTITIONED|
+                            -- SORT_GROUP_BY[$$261]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- REPLICATE  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.tasks)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/leftouterjoin/query-ASTERIXDB-2857.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/leftouterjoin/query-ASTERIXDB-2857.plan
index 18934ba..79d50d8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/leftouterjoin/query-ASTERIXDB-2857.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/leftouterjoin/query-ASTERIXDB-2857.plan
@@ -1,39 +1,34 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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|
-                  -- NESTED_LOOP  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |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|
-                                      -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+    -- ASSIGN  |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|
+                -- NESTED_LOOP  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$136][$$137]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$136]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |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 e8d4f16..d7a81cb 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
@@ -1,20 +1,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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|
-                    -- 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|
-                                  -- DATASOURCE_SCAN (test.DS1)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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|
+                  -- 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|
+                                -- 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/meta/indexes_on_dataset_with_meta_08.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/meta/indexes_on_dataset_with_meta_08.plan
index 2561ec6..9dbe5c2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/meta/indexes_on_dataset_with_meta_08.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/meta/indexes_on_dataset_with_meta_08.plan
@@ -1,10 +1,9 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_SELECT  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- DATASOURCE_SCAN (test.DS1)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |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 9d2ca53..9887b10 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
@@ -1,15 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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|
-              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
-                -- DATASOURCE_SCAN (Facebook.Friendship)  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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|
+            -- 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/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.plan
index 9ae6c48..6af09d7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.plan
@@ -1,30 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$48(ASC), $$49(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$48(ASC), $$49(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- HYBRID_HASH_JOIN [$$45][$$46]  |PARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$45]  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$46]  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$48(ASC), $$49(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$48(ASC), $$49(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- HYBRID_HASH_JOIN [$$45][$$46]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$45]  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$46]  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/03.plan
index d874441..e831481 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/03.plan
@@ -1,15 +1,13 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$22(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/04.plan
index 58cd521..a90a4f1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/04.plan
@@ -1,17 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/05.plan
index 58cd521..a90a4f1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/05.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/05.plan
@@ -1,17 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/06.plan
index 58cd521..a90a4f1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/06.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/06.plan
@@ -1,17 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/07.plan
index 58cd521..a90a4f1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/07.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/07.plan
@@ -1,17 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/08.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/08.plan
index 58cd521..a90a4f1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/08.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/08.plan
@@ -1,17 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/09.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/09.plan
index 58cd521..a90a4f1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/09.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/09.plan
@@ -1,17 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/10.plan
index 58cd521..a90a4f1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/10.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/10.plan
@@ -1,17 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/11.plan
index bbd2dba..eacac4b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/11.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/11.plan
@@ -1,18 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$29(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$30(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/12.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/12.plan
index fde8db4..eacac4b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/12.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index/non-enforced-composite-key/12.plan
@@ -1,17 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$29(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$30(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/inverted-index-join/ngram-contains_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/inverted-index-join/ngram-contains_01_ps.plan
index 84cd82b..523165d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/inverted-index-join/ngram-contains_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/inverted-index-join/ngram-contains_01_ps.plan
@@ -1,44 +1,39 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
-              -- RANGE_PARTITION_EXCHANGE [$$40(ASC), $$41(ASC)]  |PARTITIONED|
-                -- FORWARD  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- REPLICATE  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |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|
+                                              -- 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/nested-open-index/inverted-index-join/ngram-contains_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/inverted-index-join/ngram-contains_02_ps.plan
index 74fe06d..a48bd9a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/inverted-index-join/ngram-contains_02_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/inverted-index-join/ngram-contains_02_ps.plan
@@ -1,44 +1,39 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
-              -- RANGE_PARTITION_EXCHANGE [$$40(ASC), $$41(ASC)]  |PARTITIONED|
-                -- FORWARD  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- REPLICATE  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (test.DBLP.DBLP)  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.CSX)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested_loj2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested_loj2.plan
index 15b8d6c..f607474 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested_loj2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested_loj2.plan
@@ -1,43 +1,42 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- PRE_CLUSTERED_GROUP_BY[$$71]  |PARTITIONED|
-                    {
-                      -- AGGREGATE  |LOCAL|
-                        -- MICRO_PRE_CLUSTERED_GROUP_BY[$$72]  |LOCAL|
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- STREAM_SELECT  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
-                          -- STREAM_SELECT  |LOCAL|
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                    }
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$71(ASC), $$72(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- HYBRID_HASH_JOIN [$$72][$$73]  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |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 [$$75]  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$71]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- MICRO_PRE_CLUSTERED_GROUP_BY[$$72]  |LOCAL|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- STREAM_SELECT  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$71(ASC), $$72(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$72][$$73]  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$71][$$75]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$71]  |PARTITIONED|
+                              -- DATASOURCE_SCAN (tpch.Customers)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (tpch.LineItems)  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |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|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN (tpch.LineItems)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-enforced/inverted-index-join/ngram-contains_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-enforced/inverted-index-join/ngram-contains_01_ps.plan
index 84cd82b..523165d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-enforced/inverted-index-join/ngram-contains_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-enforced/inverted-index-join/ngram-contains_01_ps.plan
@@ -1,44 +1,39 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
-              -- RANGE_PARTITION_EXCHANGE [$$40(ASC), $$41(ASC)]  |PARTITIONED|
-                -- FORWARD  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- REPLICATE  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.DBLP)  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (test.CSX.CSX)  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |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|
+                                              -- 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/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-03.plan
index f1a1616..3af30be 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-03.plan
@@ -1,10 +1,9 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-04.plan
index f1a1616..3af30be 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-04.plan
@@ -1,10 +1,9 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07.plan
index 8d30fd0..09a9501 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07.plan
@@ -1,26 +1,23 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$40(ASC), $$41(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- HYBRID_HASH_JOIN [$$37][$$38]  |PARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$40(ASC), $$41(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- HYBRID_HASH_JOIN [$$37][$$38]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.plan
index 6c25f6d..df1a86f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.plan
@@ -1,56 +1,51 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$40(ASC), $$41(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 [$$37][$$38]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$40(ASC), $$41(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 [$$37][$$38]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$37][$$38]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$37][$$38]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (test.TestOpen2)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04.plan
index 0b23e02..735ca65 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04.plan
@@ -1,15 +1,13 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.plan
index 8d94902..953f6e6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.plan
@@ -1,34 +1,31 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$20(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$21(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05.plan
index 0b23e02..735ca65 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05.plan
@@ -1,15 +1,13 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.plan
index 8d94902..953f6e6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.plan
@@ -1,34 +1,31 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$20(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$21(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06.plan
index 0b23e02..735ca65 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06.plan
@@ -1,15 +1,13 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.plan
index 8d94902..953f6e6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.plan
@@ -1,34 +1,31 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$20(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$21(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07.plan
index 0b23e02..735ca65 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07.plan
@@ -1,15 +1,13 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.plan
index 8d94902..953f6e6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.plan
@@ -1,34 +1,31 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$20(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$21(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08.plan
index 0b23e02..735ca65 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08.plan
@@ -1,15 +1,13 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.plan
index 8d94902..953f6e6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.plan
@@ -1,34 +1,31 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$20(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$21(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09.plan
index 0b23e02..735ca65 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09.plan
@@ -1,15 +1,13 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.plan
index 8d94902..953f6e6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.plan
@@ -1,34 +1,31 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$20(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$21(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10.plan
index 0b23e02..735ca65 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10.plan
@@ -1,15 +1,13 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105.plan
index c1080c7..8c3f7ca 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105.plan
@@ -1,15 +1,13 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.plan
index be79cf5..54d9469 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.plan
@@ -1,34 +1,31 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$24(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$25(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.plan
index 8d94902..953f6e6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.plan
@@ -1,34 +1,31 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$20(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$21(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11.plan
index 748b085..efb563b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11.plan
@@ -1,16 +1,13 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.plan
index 35ffdd2..4df11b8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.plan
@@ -1,36 +1,31 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$23(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- REPLICATE  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$24(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test.TestOpen)  |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 cf73858..623713a 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
@@ -13,15 +13,13 @@
                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_01_ps.plan
index 8e1370f..bab6397 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_01_ps.plan
@@ -13,34 +13,31 @@
                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$24(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$24(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |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 b57024b..1ea9571 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
@@ -13,15 +13,13 @@
                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$26(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$26(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_02_ps.plan
index 6db1de8..d6fbaae 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_02_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_02_ps.plan
@@ -13,34 +13,31 @@
                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$26(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$26(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |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 cf73858..623713a 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
@@ -13,15 +13,13 @@
                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_01_ps.plan
index 8e1370f..bab6397 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_01_ps.plan
@@ -13,34 +13,31 @@
                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$24(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$24(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |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 b57024b..1ea9571 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
@@ -13,15 +13,13 @@
                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$26(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$26(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_02_ps.plan
index 6db1de8..d6fbaae 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_02_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_02_ps.plan
@@ -13,34 +13,31 @@
                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$26(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$26(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |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 21c0576..9f0d653 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
@@ -1,78 +1,70 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$230][$$212]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- HYBRID_HASH_JOIN [$$216][$$211]  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$218][$$210]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- HYBRID_HASH_JOIN [$$221][$$209]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- HYBRID_HASH_JOIN [$$206][$$208]  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |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 [$$233]  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- DATASOURCE_SCAN (q08_group_by.LineItem)  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_SELECT  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (q08_group_by.Orders)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                      -- BROADCAST_EXCHANGE  |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|
-                              -- BROADCAST_EXCHANGE  |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|
-                      -- BROADCAST_EXCHANGE  |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|
-              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- HYBRID_HASH_JOIN [$$230][$$212]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$216][$$211]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (q08_group_by.Part)  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$218][$$210]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$221][$$209]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$206][$$208]  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- HYBRID_HASH_JOIN [$$205][$$233]  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$205]  |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 [$$233]  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (q08_group_by.LineItem)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (q08_group_by.Orders)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (q08_group_by.Customer)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (q08_group_by.Nation)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- BROADCAST_EXCHANGE  |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|
+            -- BROADCAST_EXCHANGE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (q08_group_by.Part)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |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 cc7e6cc..0e70665 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
@@ -1,55 +1,49 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$157][$$151]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- HYBRID_HASH_JOIN [$$154][$$150]  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$169][$$147]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$169]  |PARTITIONED|
-                                -- STREAM_PROJECT  |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 [$$169, $$154]  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN (q09_group_by.LineItem)  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |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|
-                      -- BROADCAST_EXCHANGE  |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|
-              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- HYBRID_HASH_JOIN [$$157][$$151]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$154][$$150]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (q09_group_by.Nation)  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$169][$$147]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$169]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$148, $$149][$$169, $$154]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$148, $$149]  |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 [$$169, $$154]  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN (q09_group_by.LineItem)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$147]  |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|
+                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (q09_group_by.Supplier)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- BROADCAST_EXCHANGE  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (q09_group_by.Nation)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query-ASTERIXDB-2700.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query-ASTERIXDB-2700.plan
index 5c72fa2..1f5b867 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query-ASTERIXDB-2700.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query-ASTERIXDB-2700.plan
@@ -1,28 +1,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (bigfun.GleambookMessagesComposite.GleambookMessagesComposite)  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- BTREE_SEARCH (bigfun.GleambookMessagesComposite.GleambookMessagesComposite)  |PARTITIONED|
+                -- STABLE_SORT [$$58(ASC), $$59(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$58(ASC), $$59(ASC)]  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH (bigfun.GleambookMessagesComposite.authorIdIx)  |PARTITIONED|
-                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- SORT_MERGE_EXCHANGE [$$48(ASC) ]  |PARTITIONED|
-                                    -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (bigfun.GleambookMessagesComposite.authorIdIx)  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- SORT_MERGE_EXCHANGE [$$48(ASC) ]  |PARTITIONED|
+                                -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
-                                          -- STREAM_SELECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- DATASOURCE_SCAN (bigfun.GleambookUsersComposite)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN (bigfun.GleambookUsersComposite)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query-ASTERIXDB-3334.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query-ASTERIXDB-3334.plan
index bbb94e3..54dec5e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query-ASTERIXDB-3334.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query-ASTERIXDB-3334.plan
@@ -1,61 +1,56 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- SORT_GROUP_BY[$$1112]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$1112]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$1113]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$1113, $$1114][$$Employee Name, $$Call Center Region]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$1114, $$1113]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- ASSIGN  |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 (test.collection0)  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- SORT_GROUP_BY[$$1120]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$1120]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$1121]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$1121, $$1122][$$Employee Name, $$Call Center Region]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$1122, $$1121]  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- SORT_GROUP_BY[$$1109, $$1110]  |PARTITIONED|
+                                -- REPLICATE  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- SORT_GROUP_BY[$$1117, $$1118]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- HASH_PARTITION_EXCHANGE [$$1117, $$1118]  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$1125, $$1124]  |PARTITIONED|
                                         {
                                           -- AGGREGATE  |LOCAL|
                                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                                         }
-                                  -- HASH_PARTITION_EXCHANGE [$$1109, $$1110]  |PARTITIONED|
-                                    -- SORT_GROUP_BY[$$1117, $$1116]  |PARTITIONED|
-                                            {
-                                              -- AGGREGATE  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                            }
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- STREAM_SELECT  |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 (test.collection0)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_SELECT  |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 (test.collection0)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query_issue3316.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query_issue3316.plan
index 96df2d9..3d62392 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query_issue3316.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query_issue3316.plan
@@ -4,474 +4,80 @@
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- STABLE_SORT [$$192(ASC)]  |PARTITIONED|
           -- HASH_PARTITION_EXCHANGE [$$192]  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- PRE_CLUSTERED_GROUP_BY[$$227]  |PARTITIONED|
-                            {
-                              -- 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 [$$227][$$388]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$193]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- STREAM_SELECT  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$193(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- HYBRID_HASH_JOIN [$$193][$$240]  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$193]  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- PRE_CLUSTERED_GROUP_BY[$$227]  |PARTITIONED|
+                          {
+                            -- 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 [$$227][$$388]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$193]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$193(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$193][$$240]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$193]  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$240]  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$240]  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
                                                 -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$126][$$199]  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- HYBRID_HASH_JOIN [$$126][$$199]  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- STREAM_SELECT  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- PRE_CLUSTERED_GROUP_BY[$$225, $$194]  |PARTITIONED|
-                                                                            {
-                                                                              -- AGGREGATE  |LOCAL|
-                                                                                -- STREAM_SELECT  |LOCAL|
-                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                            }
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- STABLE_SORT [$$225(ASC), $$194(ASC)]  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- HYBRID_HASH_JOIN [$$225, $$194][$$254, $$255]  |PARTITIONED|
-                                                                                  -- HASH_PARTITION_EXCHANGE [$$225, $$194]  |PARTITIONED|
-                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- HYBRID_HASH_JOIN [$$223][$$207]  |PARTITIONED|
-                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- PRE_CLUSTERED_GROUP_BY[$$225, $$194]  |PARTITIONED|
+                                                                      {
+                                                                        -- AGGREGATE  |LOCAL|
+                                                                          -- STREAM_SELECT  |LOCAL|
+                                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                      }
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- STABLE_SORT [$$225(ASC), $$194(ASC)]  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- HYBRID_HASH_JOIN [$$225, $$194][$$254, $$255]  |PARTITIONED|
+                                                                            -- HASH_PARTITION_EXCHANGE [$$225, $$194]  |PARTITIONED|
+                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- HYBRID_HASH_JOIN [$$223][$$207]  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                            -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
                                                                                               -- ASSIGN  |PARTITIONED|
                                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                   -- REPLICATE  |PARTITIONED|
-                                                                                                    -- HASH_PARTITION_EXCHANGE [$$445]  |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|
-                                                                                                                      -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
-                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                          -- HASH_PARTITION_EXCHANGE [$$207]  |PARTITIONED|
-                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                              -- STREAM_SELECT  |PARTITIONED|
-                                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                        -- REPLICATE  |PARTITIONED|
-                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                            -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                        -- STREAM_SELECT  |PARTITIONED|
-                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                            -- UNNEST  |PARTITIONED|
-                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                -- PRE_CLUSTERED_GROUP_BY[$$238, $$235]  |PARTITIONED|
-                                                                                                        {
-                                                                                                          -- AGGREGATE  |LOCAL|
-                                                                                                            -- AGGREGATE  |LOCAL|
-                                                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                        }
-                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                    -- STABLE_SORT [$$238(ASC), $$235(ASC)]  |PARTITIONED|
-                                                                                                      -- HASH_PARTITION_EXCHANGE [$$238, $$235]  |PARTITIONED|
-                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                            -- HYBRID_HASH_JOIN [$$249, $$229][$$235, $$228]  |PARTITIONED|
-                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                    -- HYBRID_HASH_JOIN [$$243][$$242]  |PARTITIONED|
-                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                            -- HYBRID_HASH_JOIN [$$252][$$238]  |PARTITIONED|
-                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                    -- HYBRID_HASH_JOIN [$$210][$$243]  |PARTITIONED|
-                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$210]  |PARTITIONED|
-                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                            -- HYBRID_HASH_JOIN [$$135][$$198]  |PARTITIONED|
-                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                  -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                    -- ASSIGN  |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 (test.collection1)  |PARTITIONED|
-                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                                                -- UNNEST  |UNPARTITIONED|
-                                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                          -- ASSIGN  |PARTITIONED|
-                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                              -- REPLICATE  |PARTITIONED|
-                                                                                                                                                -- HASH_PARTITION_EXCHANGE [$$445]  |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|
-                                                                                                                                                                  -- DATASOURCE_SCAN (test.collection0)  |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|
-                                                                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                                          -- REPLICATE  |PARTITIONED|
-                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                    -- DATASOURCE_SCAN (test.collection0)  |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|
-                                                                                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                        -- REPLICATE  |PARTITIONED|
-                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                            -- ASSIGN  |PARTITIONED|
-                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                    -- REPLICATE  |PARTITIONED|
-                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                        -- DATASOURCE_SCAN (test.collection1)  |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|
-                                                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                            -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                -- REPLICATE  |PARTITIONED|
-                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                        -- ASSIGN  |PARTITIONED|
-                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                            -- REPLICATE  |PARTITIONED|
-                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                            -- UNNEST  |UNPARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$388]  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- HYBRID_HASH_JOIN [$$162][$$202]  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- HYBRID_HASH_JOIN [$$165][$$201]  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- PRE_CLUSTERED_GROUP_BY[$$256, $$196]  |PARTITIONED|
-                                                                    {
-                                                                      -- AGGREGATE  |LOCAL|
-                                                                        -- STREAM_SELECT  |LOCAL|
-                                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                    }
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- STABLE_SORT [$$256(ASC), $$196(ASC)]  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- HYBRID_HASH_JOIN [$$256, $$196][$$459, $$460]  |PARTITIONED|
-                                                                          -- HASH_PARTITION_EXCHANGE [$$256, $$196]  |PARTITIONED|
-                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- HYBRID_HASH_JOIN [$$260][$$214]  |PARTITIONED|
-                                                                                  -- HASH_PARTITION_EXCHANGE [$$260]  |PARTITIONED|
-                                                                                    -- PRE_CLUSTERED_GROUP_BY[$$257]  |PARTITIONED|
-                                                                                            {
-                                                                                              -- AGGREGATE  |LOCAL|
-                                                                                                -- STREAM_SELECT  |LOCAL|
-                                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                            }
-                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- STABLE_SORT [$$257(ASC)]  |PARTITIONED|
-                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                -- HYBRID_HASH_JOIN [$$257][$$261]  |PARTITIONED|
-                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                       -- ASSIGN  |PARTITIONED|
                                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          -- REPLICATE  |PARTITIONED|
-                                                                                                            -- HASH_PARTITION_EXCHANGE [$$399]  |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|
-                                                                                                                              -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
-                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                  -- HASH_PARTITION_EXCHANGE [$$261]  |PARTITIONED|
-                                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          -- HYBRID_HASH_JOIN [$$266][$$265]  |PARTITIONED|
+                                                                                                          -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
                                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$268, $$270]  |PARTITIONED|
-                                                                                                                              {
-                                                                                                                                -- AGGREGATE  |LOCAL|
-                                                                                                                                  -- STREAM_SELECT  |LOCAL|
-                                                                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                                              }
-                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                          -- STABLE_SORT [$$268(ASC), $$270(ASC)]  |PARTITIONED|
-                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                  -- HYBRID_HASH_JOIN [$$268, $$270][$$278, $$279]  |PARTITIONED|
-                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                        -- ASSIGN  |PARTITIONED|
-                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                            -- REPLICATE  |PARTITIONED|
-                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$409, $$411]  |PARTITIONED|
-                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                    -- HYBRID_HASH_JOIN [$$417][$$418]  |PARTITIONED|
-                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                          -- ASSIGN  |PARTITIONED|
-                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                              -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                -- HASH_PARTITION_EXCHANGE [$$445]  |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|
-                                                                                                                                                                                  -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
-                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$418]  |PARTITIONED|
-                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                          -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                            -- ASSIGN  |PARTITIONED|
-                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                    -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                          -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                              -- UNNEST  |PARTITIONED|
-                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                  -- PRE_CLUSTERED_GROUP_BY[$$284, $$285]  |PARTITIONED|
-                                                                                                                                                          {
-                                                                                                                                                            -- AGGREGATE  |LOCAL|
-                                                                                                                                                              -- AGGREGATE  |LOCAL|
-                                                                                                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                                                                          }
-                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                      -- STABLE_SORT [$$284(ASC), $$285(ASC)]  |PARTITIONED|
-                                                                                                                                                        -- HASH_PARTITION_EXCHANGE [$$284, $$285]  |PARTITIONED|
-                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                              -- HYBRID_HASH_JOIN [$$295, $$287][$$285, $$286]  |PARTITIONED|
-                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                      -- HYBRID_HASH_JOIN [$$304][$$307]  |PARTITIONED|
-                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                              -- HYBRID_HASH_JOIN [$$294][$$284]  |PARTITIONED|
-                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                      -- HYBRID_HASH_JOIN [$$303][$$304]  |PARTITIONED|
-                                                                                                                                                                                        -- HASH_PARTITION_EXCHANGE [$$303]  |PARTITIONED|
-                                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                              -- HYBRID_HASH_JOIN [$$302][$$301]  |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|
-                                                                                                                                                                                                              -- STREAM_SELECT  |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 (test.collection1)  |PARTITIONED|
-                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                  -- UNNEST  |UNPARTITIONED|
-                                                                                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                            -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$445]  |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|
-                                                                                                                                                                                                                    -- DATASOURCE_SCAN (test.collection0)  |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|
-                                                                                                                                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                      -- DATASOURCE_SCAN (test.collection0)  |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|
-                                                                                                                                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                          -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                          -- DATASOURCE_SCAN (test.collection1)  |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|
-                                                                                                                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                              -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                              -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                  -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                              -- UNNEST  |UNPARTITIONED|
-                                                                                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                                                                  -- HASH_PARTITION_EXCHANGE [$$214]  |PARTITIONED|
-                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                    -- HASH_PARTITION_EXCHANGE [$$207]  |PARTITIONED|
+                                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                                        -- ASSIGN  |PARTITIONED|
                                                                                           -- ASSIGN  |PARTITIONED|
                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                               -- REPLICATE  |PARTITIONED|
@@ -479,479 +85,220 @@
                                                                                                   -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                       -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
                                                                                 -- STREAM_SELECT  |PARTITIONED|
-                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                    -- UNNEST  |PARTITIONED|
-                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- PRE_CLUSTERED_GROUP_BY[$$327, $$326]  |PARTITIONED|
-                                                                                                {
+                                                                                  -- UNNEST  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$238, $$235]  |PARTITIONED|
+                                                                                              {
+                                                                                                -- AGGREGATE  |LOCAL|
                                                                                                   -- AGGREGATE  |LOCAL|
-                                                                                                    -- AGGREGATE  |LOCAL|
-                                                                                                      -- STREAM_SELECT  |LOCAL|
-                                                                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                }
-                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            -- STABLE_SORT [$$327(ASC), $$326(ASC)]  |PARTITIONED|
-                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                    -- HYBRID_HASH_JOIN [$$327, $$326][$$398, $$397]  |PARTITIONED|
-                                                                                                      -- HASH_PARTITION_EXCHANGE [$$327, $$326]  |PARTITIONED|
-                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                            -- HYBRID_HASH_JOIN [$$320][$$319]  |PARTITIONED|
-                                                                                                              -- HASH_PARTITION_EXCHANGE [$$320]  |PARTITIONED|
-                                                                                                                -- PRE_CLUSTERED_GROUP_BY[$$328]  |PARTITIONED|
-                                                                                                                        {
-                                                                                                                          -- AGGREGATE  |LOCAL|
-                                                                                                                            -- STREAM_SELECT  |LOCAL|
-                                                                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                                        }
-                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                    -- STABLE_SORT [$$328(ASC)]  |PARTITIONED|
-                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                            -- HYBRID_HASH_JOIN [$$328][$$331]  |PARTITIONED|
-                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                      -- REPLICATE  |PARTITIONED|
-                                                                                                                                        -- HASH_PARTITION_EXCHANGE [$$399]  |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|
-                                                                                                                                                          -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
-                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$331]  |PARTITIONED|
-                                                                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                      -- HYBRID_HASH_JOIN [$$336][$$335]  |PARTITIONED|
-                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                            -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                  -- PRE_CLUSTERED_GROUP_BY[$$338, $$340]  |PARTITIONED|
-                                                                                                                                                          {
-                                                                                                                                                            -- AGGREGATE  |LOCAL|
-                                                                                                                                                              -- STREAM_SELECT  |LOCAL|
-                                                                                                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                                                                          }
-                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                      -- STABLE_SORT [$$338(ASC), $$340(ASC)]  |PARTITIONED|
-                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                              -- HYBRID_HASH_JOIN [$$338, $$340][$$348, $$349]  |PARTITIONED|
-                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$409, $$411]  |PARTITIONED|
-                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                -- HYBRID_HASH_JOIN [$$417][$$418]  |PARTITIONED|
-                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                          -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$445]  |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|
-                                                                                                                                                                                                              -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
-                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$418]  |PARTITIONED|
-                                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                          -- UNNEST  |PARTITIONED|
-                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                              -- PRE_CLUSTERED_GROUP_BY[$$354, $$355]  |PARTITIONED|
-                                                                                                                                                                                      {
-                                                                                                                                                                                        -- AGGREGATE  |LOCAL|
-                                                                                                                                                                                          -- AGGREGATE  |LOCAL|
-                                                                                                                                                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                                                                                                      }
-                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                  -- STABLE_SORT [$$354(ASC), $$355(ASC)]  |PARTITIONED|
-                                                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$354, $$355]  |PARTITIONED|
-                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                          -- HYBRID_HASH_JOIN [$$365, $$357][$$355, $$356]  |PARTITIONED|
-                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                  -- HYBRID_HASH_JOIN [$$374][$$377]  |PARTITIONED|
-                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                          -- HYBRID_HASH_JOIN [$$364][$$354]  |PARTITIONED|
-                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                  -- HYBRID_HASH_JOIN [$$373][$$374]  |PARTITIONED|
-                                                                                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$373]  |PARTITIONED|
-                                                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                          -- HYBRID_HASH_JOIN [$$372][$$371]  |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|
-                                                                                                                                                                                                                                          -- STREAM_SELECT  |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 (test.collection1)  |PARTITIONED|
-                                                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                              -- UNNEST  |UNPARTITIONED|
-                                                                                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$445]  |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|
-                                                                                                                                                                                                                                                -- DATASOURCE_SCAN (test.collection0)  |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|
-                                                                                                                                                                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                  -- DATASOURCE_SCAN (test.collection0)  |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|
-                                                                                                                                                                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                  -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                      -- DATASOURCE_SCAN (test.collection1)  |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|
-                                                                                                                                                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                          -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                              -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                          -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                              -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                                          -- UNNEST  |UNPARTITIONED|
-                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                                                                                              -- HASH_PARTITION_EXCHANGE [$$319]  |PARTITIONED|
-                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                    -- STREAM_SELECT  |LOCAL|
+                                                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                              }
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- STABLE_SORT [$$238(ASC), $$235(ASC)]  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- HYBRID_HASH_JOIN [$$238, $$235][$$252, $$249]  |PARTITIONED|
+                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                            -- HASH_PARTITION_EXCHANGE [$$425, $$426]  |PARTITIONED|
+                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- HYBRID_HASH_JOIN [$$428][$$427]  |PARTITIONED|
                                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                      -- ASSIGN  |PARTITIONED|
                                                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                          -- ASSIGN  |PARTITIONED|
-                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
                                                                                                                               -- ASSIGN  |PARTITIONED|
                                                                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                   -- REPLICATE  |PARTITIONED|
                                                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                      -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                      -- ASSIGN  |PARTITIONED|
                                                                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                      -- HASH_PARTITION_EXCHANGE [$$398, $$397]  |PARTITIONED|
-                                                                                                        -- ASSIGN  |PARTITIONED|
-                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                              -- HYBRID_HASH_JOIN [$$172][$$200]  |PARTITIONED|
-                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                      -- HYBRID_HASH_JOIN [$$391][$$216]  |PARTITIONED|
-                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                              -- HYBRID_HASH_JOIN [$$391][$$390]  |PARTITIONED|
-                                                                                                                                -- HASH_PARTITION_EXCHANGE [$$391]  |PARTITIONED|
-                                                                                                                                  -- PRE_CLUSTERED_GROUP_BY[$$399]  |PARTITIONED|
-                                                                                                                                          {
-                                                                                                                                            -- AGGREGATE  |LOCAL|
-                                                                                                                                              -- STREAM_SELECT  |LOCAL|
-                                                                                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                                                          }
-                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                      -- STABLE_SORT [$$399(ASC)]  |PARTITIONED|
-                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                          -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
                                                                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                              -- HYBRID_HASH_JOIN [$$399][$$402]  |PARTITIONED|
-                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                  -- REPLICATE  |PARTITIONED|
-                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$399]  |PARTITIONED|
-                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$427]  |PARTITIONED|
+                                                                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                        -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                    -- HASH_PARTITION_EXCHANGE [$$252, $$249]  |PARTITIONED|
+                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                            -- HYBRID_HASH_JOIN [$$135][$$198]  |PARTITIONED|
+                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                    -- HYBRID_HASH_JOIN [$$243][$$210]  |PARTITIONED|
+                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                    -- HYBRID_HASH_JOIN [$$445][$$448]  |PARTITIONED|
+                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
+                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                                         -- ASSIGN  |PARTITIONED|
                                                                                                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                            -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
                                                                                                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                      -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
-                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                -- HASH_PARTITION_EXCHANGE [$$402]  |PARTITIONED|
+                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$448]  |PARTITIONED|
+                                                                                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                -- ASSIGN  |PARTITIONED|
                                                                                                                                                   -- ASSIGN  |PARTITIONED|
-                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                        -- HYBRID_HASH_JOIN [$$407][$$406]  |PARTITIONED|
-                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                              -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                    -- PRE_CLUSTERED_GROUP_BY[$$409, $$411]  |PARTITIONED|
-                                                                                                                                                                            {
-                                                                                                                                                                              -- AGGREGATE  |LOCAL|
-                                                                                                                                                                                -- STREAM_SELECT  |LOCAL|
-                                                                                                                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                                                                                            }
-                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                        -- STABLE_SORT [$$409(ASC), $$411(ASC)]  |PARTITIONED|
-                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                -- HYBRID_HASH_JOIN [$$409, $$411][$$419, $$420]  |PARTITIONED|
-                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$409, $$411]  |PARTITIONED|
-                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                            -- HYBRID_HASH_JOIN [$$417][$$418]  |PARTITIONED|
-                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                        -- HASH_PARTITION_EXCHANGE [$$445]  |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|
-                                                                                                                                                                                                                          -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
-                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$418]  |PARTITIONED|
-                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                  -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                            -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                        -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                            -- UNNEST  |PARTITIONED|
-                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                -- PRE_CLUSTERED_GROUP_BY[$$425, $$426]  |PARTITIONED|
-                                                                                                                                                                                                        {
-                                                                                                                                                                                                          -- AGGREGATE  |LOCAL|
-                                                                                                                                                                                                            -- AGGREGATE  |LOCAL|
-                                                                                                                                                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                                                                                                                                        }
-                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                    -- STABLE_SORT [$$425(ASC), $$426(ASC)]  |PARTITIONED|
-                                                                                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$425, $$426]  |PARTITIONED|
-                                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                            -- HYBRID_HASH_JOIN [$$436, $$428][$$426, $$427]  |PARTITIONED|
-                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                    -- HYBRID_HASH_JOIN [$$445][$$448]  |PARTITIONED|
-                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                            -- HYBRID_HASH_JOIN [$$435][$$425]  |PARTITIONED|
-                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                    -- HYBRID_HASH_JOIN [$$444][$$445]  |PARTITIONED|
-                                                                                                                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$444]  |PARTITIONED|
-                                                                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                            -- HYBRID_HASH_JOIN [$$443][$$442]  |PARTITIONED|
-                                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                                                      -- STREAM_SELECT  |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 (test.collection1)  |PARTITIONED|
-                                                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                                                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                -- UNNEST  |UNPARTITIONED|
-                                                                                                                                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$445]  |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|
-                                                                                                                                                                                                                                                            -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
-                                                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                              -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
-                                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                              -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                              -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                  -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                          -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
-                                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                          -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
-                                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                                                            -- UNNEST  |UNPARTITIONED|
-                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                                                                                                                -- HASH_PARTITION_EXCHANGE [$$390]  |PARTITIONED|
-                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                          -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                     -- ASSIGN  |PARTITIONED|
                                                                                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                         -- REPLICATE  |PARTITIONED|
                                                                                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                            -- ASSIGN  |PARTITIONED|
-                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                            -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                -- UNNEST  |UNPARTITIONED|
+                                                                                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- UNNEST  |UNPARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$388]  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$162][$$202]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$165][$$201]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_SELECT  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- PRE_CLUSTERED_GROUP_BY[$$256, $$196]  |PARTITIONED|
+                                                              {
+                                                                -- AGGREGATE  |LOCAL|
+                                                                  -- STREAM_SELECT  |LOCAL|
+                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                              }
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- STABLE_SORT [$$256(ASC), $$196(ASC)]  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- HYBRID_HASH_JOIN [$$256, $$196][$$459, $$460]  |PARTITIONED|
+                                                                    -- HASH_PARTITION_EXCHANGE [$$256, $$196]  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- HYBRID_HASH_JOIN [$$260][$$214]  |PARTITIONED|
+                                                                            -- HASH_PARTITION_EXCHANGE [$$260]  |PARTITIONED|
+                                                                              -- PRE_CLUSTERED_GROUP_BY[$$257]  |PARTITIONED|
+                                                                                      {
+                                                                                        -- AGGREGATE  |LOCAL|
+                                                                                          -- STREAM_SELECT  |LOCAL|
+                                                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                      }
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- STABLE_SORT [$$257(ASC)]  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- HYBRID_HASH_JOIN [$$257][$$261]  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                    -- HASH_PARTITION_EXCHANGE [$$399]  |PARTITIONED|
+                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                            -- HASH_PARTITION_EXCHANGE [$$261]  |PARTITIONED|
+                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- HYBRID_HASH_JOIN [$$266][$$265]  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- PRE_CLUSTERED_GROUP_BY[$$268, $$270]  |PARTITIONED|
+                                                                                                                      {
+                                                                                                                        -- AGGREGATE  |LOCAL|
+                                                                                                                          -- STREAM_SELECT  |LOCAL|
+                                                                                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                      }
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- STABLE_SORT [$$268(ASC), $$270(ASC)]  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- HYBRID_HASH_JOIN [$$268, $$270][$$278, $$279]  |PARTITIONED|
+                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$409, $$411]  |PARTITIONED|
+                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- HYBRID_HASH_JOIN [$$417][$$418]  |PARTITIONED|
+                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                                                -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
+                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                              -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$418]  |PARTITIONED|
+                                                                                                                                              -- STREAM_SELECT  |PARTITIONED|
                                                                                                                                                 -- ASSIGN  |PARTITIONED|
                                                                                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                                     -- REPLICATE  |PARTITIONED|
@@ -959,25 +306,595 @@
                                                                                                                                                         -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
                                                                                                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                        -- HASH_PARTITION_EXCHANGE [$$216]  |PARTITIONED|
-                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                            -- ASSIGN  |PARTITIONED|
-                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                  -- UNNEST  |PARTITIONED|
+                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$284, $$285]  |PARTITIONED|
+                                                                                                                                              {
+                                                                                                                                                -- AGGREGATE  |LOCAL|
+                                                                                                                                                  -- AGGREGATE  |LOCAL|
+                                                                                                                                                    -- STREAM_SELECT  |LOCAL|
+                                                                                                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                              }
+                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- STABLE_SORT [$$284(ASC), $$285(ASC)]  |PARTITIONED|
+                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                  -- HYBRID_HASH_JOIN [$$284, $$285][$$294, $$295]  |PARTITIONED|
+                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$425, $$426]  |PARTITIONED|
+                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                  -- HYBRID_HASH_JOIN [$$428][$$427]  |PARTITIONED|
+                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
+                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                          -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$427]  |PARTITIONED|
+                                                                                                                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                        -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$294, $$295]  |PARTITIONED|
+                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                            -- HYBRID_HASH_JOIN [$$302][$$301]  |PARTITIONED|
+                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                    -- HYBRID_HASH_JOIN [$$304][$$303]  |PARTITIONED|
+                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                    -- HYBRID_HASH_JOIN [$$445][$$448]  |PARTITIONED|
+                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
+                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                            -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$448]  |PARTITIONED|
+                                                                                                                                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                          -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                  -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                -- UNNEST  |UNPARTITIONED|
+                                                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                        -- UNNEST  |UNPARTITIONED|
+                                                                                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                            -- HASH_PARTITION_EXCHANGE [$$214]  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                          -- UNNEST  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- PRE_CLUSTERED_GROUP_BY[$$327, $$326]  |PARTITIONED|
+                                                                                      {
+                                                                                        -- AGGREGATE  |LOCAL|
+                                                                                          -- AGGREGATE  |LOCAL|
+                                                                                            -- STREAM_SELECT  |LOCAL|
+                                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                      }
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- STABLE_SORT [$$327(ASC), $$326(ASC)]  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- HYBRID_HASH_JOIN [$$327, $$326][$$398, $$397]  |PARTITIONED|
+                                                                                            -- HASH_PARTITION_EXCHANGE [$$327, $$326]  |PARTITIONED|
+                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- HYBRID_HASH_JOIN [$$320][$$319]  |PARTITIONED|
+                                                                                                    -- HASH_PARTITION_EXCHANGE [$$320]  |PARTITIONED|
+                                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$328]  |PARTITIONED|
+                                                                                                              {
+                                                                                                                -- AGGREGATE  |LOCAL|
+                                                                                                                  -- STREAM_SELECT  |LOCAL|
+                                                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                              }
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- STABLE_SORT [$$328(ASC)]  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- HYBRID_HASH_JOIN [$$328][$$331]  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$399]  |PARTITIONED|
+                                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                      -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$331]  |PARTITIONED|
+                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                            -- HYBRID_HASH_JOIN [$$336][$$335]  |PARTITIONED|
+                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$338, $$340]  |PARTITIONED|
+                                                                                                                                              {
+                                                                                                                                                -- AGGREGATE  |LOCAL|
+                                                                                                                                                  -- STREAM_SELECT  |LOCAL|
+                                                                                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                              }
+                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- STABLE_SORT [$$338(ASC), $$340(ASC)]  |PARTITIONED|
+                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                  -- HYBRID_HASH_JOIN [$$338, $$340][$$348, $$349]  |PARTITIONED|
+                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$409, $$411]  |PARTITIONED|
+                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                  -- HYBRID_HASH_JOIN [$$417][$$418]  |PARTITIONED|
+                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                        -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
+                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                      -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$418]  |PARTITIONED|
+                                                                                                                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                          -- UNNEST  |PARTITIONED|
+                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                              -- PRE_CLUSTERED_GROUP_BY[$$354, $$355]  |PARTITIONED|
+                                                                                                                                                                      {
+                                                                                                                                                                        -- AGGREGATE  |LOCAL|
+                                                                                                                                                                          -- AGGREGATE  |LOCAL|
+                                                                                                                                                                            -- STREAM_SELECT  |LOCAL|
+                                                                                                                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                                                      }
+                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                  -- STABLE_SORT [$$354(ASC), $$355(ASC)]  |PARTITIONED|
+                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                          -- HYBRID_HASH_JOIN [$$354, $$355][$$364, $$365]  |PARTITIONED|
+                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$425, $$426]  |PARTITIONED|
+                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                          -- HYBRID_HASH_JOIN [$$428][$$427]  |PARTITIONED|
+                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
+                                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                  -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$427]  |PARTITIONED|
+                                                                                                                                                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$364, $$365]  |PARTITIONED|
+                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                    -- HYBRID_HASH_JOIN [$$372][$$371]  |PARTITIONED|
+                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                            -- HYBRID_HASH_JOIN [$$374][$$373]  |PARTITIONED|
+                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                            -- HYBRID_HASH_JOIN [$$445][$$448]  |PARTITIONED|
+                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
+                                                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                    -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$448]  |PARTITIONED|
+                                                                                                                                                                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                  -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                          -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                        -- UNNEST  |UNPARTITIONED|
+                                                                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                -- UNNEST  |UNPARTITIONED|
+                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                                    -- HASH_PARTITION_EXCHANGE [$$319]  |PARTITIONED|
+                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                        -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                            -- HASH_PARTITION_EXCHANGE [$$398, $$397]  |PARTITIONED|
+                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- HYBRID_HASH_JOIN [$$172][$$200]  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                            -- HYBRID_HASH_JOIN [$$391][$$216]  |PARTITIONED|
+                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                    -- HYBRID_HASH_JOIN [$$391][$$390]  |PARTITIONED|
+                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$391]  |PARTITIONED|
+                                                                                                                        -- PRE_CLUSTERED_GROUP_BY[$$399]  |PARTITIONED|
+                                                                                                                                {
+                                                                                                                                  -- AGGREGATE  |LOCAL|
+                                                                                                                                    -- STREAM_SELECT  |LOCAL|
+                                                                                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                }
+                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                            -- STABLE_SORT [$$399(ASC)]  |PARTITIONED|
+                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
                                                                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                    -- HYBRID_HASH_JOIN [$$399][$$402]  |PARTITIONED|
                                                                                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                            -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$399]  |PARTITIONED|
+                                                                                                                                            -- REPLICATE  |PARTITIONED|
                                                                                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                  -- UNNEST  |UNPARTITIONED|
-                                                                                                                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                    -- UNNEST  |UNPARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                            -- UNNEST  |UNPARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                    -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$402]  |PARTITIONED|
+                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                              -- HYBRID_HASH_JOIN [$$407][$$406]  |PARTITIONED|
+                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                        -- PRE_CLUSTERED_GROUP_BY[$$409, $$411]  |PARTITIONED|
+                                                                                                                                                                {
+                                                                                                                                                                  -- AGGREGATE  |LOCAL|
+                                                                                                                                                                    -- STREAM_SELECT  |LOCAL|
+                                                                                                                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                                                }
+                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                            -- STABLE_SORT [$$409(ASC), $$411(ASC)]  |PARTITIONED|
+                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                    -- HYBRID_HASH_JOIN [$$409, $$411][$$419, $$420]  |PARTITIONED|
+                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$409, $$411]  |PARTITIONED|
+                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                -- HYBRID_HASH_JOIN [$$417][$$418]  |PARTITIONED|
+                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
+                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                    -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$418]  |PARTITIONED|
+                                                                                                                                                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                              -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                            -- UNNEST  |PARTITIONED|
+                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                -- PRE_CLUSTERED_GROUP_BY[$$425, $$426]  |PARTITIONED|
+                                                                                                                                                                                        {
+                                                                                                                                                                                          -- AGGREGATE  |LOCAL|
+                                                                                                                                                                                            -- AGGREGATE  |LOCAL|
+                                                                                                                                                                                              -- STREAM_SELECT  |LOCAL|
+                                                                                                                                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                                                                        }
+                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                    -- STABLE_SORT [$$425(ASC), $$426(ASC)]  |PARTITIONED|
+                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                            -- HYBRID_HASH_JOIN [$$425, $$426][$$435, $$436]  |PARTITIONED|
+                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$425, $$426]  |PARTITIONED|
+                                                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                        -- HYBRID_HASH_JOIN [$$428][$$427]  |PARTITIONED|
+                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
+                                                                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$427]  |PARTITIONED|
+                                                                                                                                                                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                              -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$435, $$436]  |PARTITIONED|
+                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                      -- HYBRID_HASH_JOIN [$$443][$$442]  |PARTITIONED|
+                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                              -- HYBRID_HASH_JOIN [$$445][$$444]  |PARTITIONED|
+                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                          -- HYBRID_HASH_JOIN [$$445][$$448]  |PARTITIONED|
+                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
+                                                                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                  -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$448]  |PARTITIONED|
+                                                                                                                                                                                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                        -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                          -- UNNEST  |UNPARTITIONED|
+                                                                                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                                  -- UNNEST  |UNPARTITIONED|
+                                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$390]  |PARTITIONED|
+                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                              -- HASH_PARTITION_EXCHANGE [$$216]  |PARTITIONED|
+                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                  -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                        -- UNNEST  |UNPARTITIONED|
+                                                                                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        -- UNNEST  |UNPARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query_issue849-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query_issue849-2.plan
index 21bb457..a73a314 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query_issue849-2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query_issue849-2.plan
@@ -16,17 +16,15 @@
               -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$47][$$49]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- DATASOURCE_SCAN (test.s)  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |UNPARTITIONED|
-                  -- ASSIGN  |UNPARTITIONED|
-                    -- UNNEST  |UNPARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- HYBRID_HASH_JOIN [$$47][$$49]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$47]  |PARTITIONED|
+              -- DATASOURCE_SCAN (test.s)  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$49]  |PARTITIONED|
+              -- ASSIGN  |UNPARTITIONED|
+                -- UNNEST  |UNPARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query_issue849.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query_issue849.plan
index 59b09a9..131d733 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query_issue849.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query_issue849.plan
@@ -16,14 +16,17 @@
               -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- BTREE_SEARCH (test.s.s)  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$55(ASC)]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$55]  |PARTITIONED|
-                    -- ASSIGN  |UNPARTITIONED|
-                      -- UNNEST  |UNPARTITIONED|
-                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- HYBRID_HASH_JOIN [$$48][$$47]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
+              -- ASSIGN  |UNPARTITIONED|
+                -- UNNEST  |UNPARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$47]  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |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_cbo/rtree-index-geometry/rtree-sidx-idxonly-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-01.plan
index 7cd72c2..e981647 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-01.plan
@@ -1,8 +1,7 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-02.plan
index 7cd72c2..e981647 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-02.plan
@@ -1,8 +1,7 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-03.plan
index 7cd72c2..e981647 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-03.plan
@@ -1,8 +1,7 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-04.plan
index 7cd72c2..e981647 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-04.plan
@@ -1,8 +1,7 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-05.plan
index 7cd72c2..e981647 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-05.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-05.plan
@@ -1,8 +1,7 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-06.plan
index 7cd72c2..e981647 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-06.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-06.plan
@@ -1,8 +1,7 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-07.plan
index 7cd72c2..e981647 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-07.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index-geometry/rtree-sidx-idxonly-07.plan
@@ -1,8 +1,7 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- DATASOURCE_SCAN (IndexGeoJSON.Geometries)  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index/rtree-sidx-idxonly-01-disable-idxonly.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index/rtree-sidx-idxonly-01-disable-idxonly.plan
index 5b57d41..c885e05 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index/rtree-sidx-idxonly-01-disable-idxonly.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index/rtree-sidx-idxonly-01-disable-idxonly.plan
@@ -1,13 +1,12 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- DATASOURCE_SCAN (test.Fragile_raw)  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.Fragile_raw)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index/rtree-sidx-idxonly-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index/rtree-sidx-idxonly-01.plan
index 5b57d41..c885e05 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index/rtree-sidx-idxonly-01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/rtree-index/rtree-sidx-idxonly-01.plan
@@ -1,13 +1,12 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      -- ASSIGN  |UNPARTITIONED|
-        -- AGGREGATE  |UNPARTITIONED|
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            -- AGGREGATE  |PARTITIONED|
-              -- STREAM_SELECT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- DATASOURCE_SCAN (test.Fragile_raw)  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |UNPARTITIONED|
+      -- AGGREGATE  |UNPARTITIONED|
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          -- AGGREGATE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.Fragile_raw)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/skip-index/skip-secondary-btree-index-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/skip-index/skip-secondary-btree-index-2.plan
index 124330a..c823a61 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/skip-index/skip-secondary-btree-index-2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/skip-index/skip-secondary-btree-index-2.plan
@@ -1,10 +1,9 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- DATASOURCE_SCAN (test.testdst)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN (test.testdst)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/split-materialization.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/split-materialization.plan
index 104e755..0a7b276 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/split-materialization.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/split-materialization.plan
@@ -1,20 +1,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$97(ASC), $$98(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$97(ASC), $$98(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (TinySocial.FacebookUsers.FacebookUsers)  |PARTITIONED|
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- STREAM_SELECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (TinySocial.FacebookUsers)  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$97(ASC), $$98(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$97(ASC), $$98(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (TinySocial.FacebookUsers.FacebookUsers)  |PARTITIONED|
+                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (TinySocial.FacebookUsers)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/split-materialization_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/split-materialization_ps.plan
index 0fdad7d..57bd09d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/split-materialization_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/split-materialization_ps.plan
@@ -1,45 +1,40 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$97(ASC), $$98(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$97(ASC), $$98(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- BTREE_SEARCH (TinySocial.FacebookUsers.FacebookUsers)  |PARTITIONED|
-                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- STREAM_SELECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN (TinySocial.FacebookUsers)  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$97(ASC), $$98(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$97(ASC), $$98(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TinySocial.FacebookUsers.FacebookUsers)  |PARTITIONED|
+                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (TinySocial.FacebookUsers)  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (TinySocial.FacebookUsers.FacebookUsers)  |PARTITIONED|
-                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                -- ASSIGN  |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|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (TinySocial.FacebookUsers.FacebookUsers)  |PARTITIONED|
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (TinySocial.FacebookUsers)  |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-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-01.plan
index 904a882..ea8f0d5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-01.plan
@@ -1,10 +1,8 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-02.plan
index 904a882..ea8f0d5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-02.plan
@@ -1,10 +1,8 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-03.plan
index 904a882..ea8f0d5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-03.plan
@@ -1,10 +1,8 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- DATASOURCE_SCAN (test.TestOpen)  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- EMPTY_TUPLE_SOURCE  |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 3bb3b7d..dd532c2 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
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$30(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.employee)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.employee)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |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 863378a..ae51cc2 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
@@ -1,14 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |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|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$31(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.employee)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- DATASOURCE_SCAN (test.employee)  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/subquery/in_as_or_2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/subquery/in_as_or_2.plan
index d6e9fa9..ed5067c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/subquery/in_as_or_2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/subquery/in_as_or_2.plan
@@ -1,28 +1,26 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- SORT_MERGE_EXCHANGE [$$26(ASC) ]  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$22]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- STREAM_SELECT  |LOCAL|
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$23][$#1]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$22]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- STREAM_SELECT  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$23][$#1]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$23]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                          -- ASSIGN  |UNPARTITIONED|
-                            -- UNNEST  |UNPARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                        -- ASSIGN  |UNPARTITIONED|
+                          -- UNNEST  |UNPARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/subquery/in_as_or_2_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/subquery/in_as_or_2_ps.plan
index de7e790..d587010 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/subquery/in_as_or_2_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/subquery/in_as_or_2_ps.plan
@@ -6,32 +6,30 @@
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- REPLICATE  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$22]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- STREAM_SELECT  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$23][$#1]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$22]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- STREAM_SELECT  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$23][$#1]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$23]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |UNPARTITIONED|
-                                        -- UNNEST  |UNPARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                                    -- ASSIGN  |UNPARTITIONED|
+                                      -- UNNEST  |UNPARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
           -- BROADCAST_EXCHANGE  |PARTITIONED|
             -- AGGREGATE  |UNPARTITIONED|
               -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
@@ -39,29 +37,27 @@
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- REPLICATE  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- STREAM_SELECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_CLUSTERED_GROUP_BY[$$22]  |PARTITIONED|
-                                      {
-                                        -- AGGREGATE  |LOCAL|
-                                          -- STREAM_SELECT  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                      }
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- HYBRID_HASH_JOIN [$$23][$#1]  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_CLUSTERED_GROUP_BY[$$22]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$23][$#1]  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$23]  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |UNPARTITIONED|
-                                                -- UNNEST  |UNPARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                                            -- ASSIGN  |UNPARTITIONED|
+                                              -- UNNEST  |UNPARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/subquery/in_as_or_4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/subquery/in_as_or_4.plan
index 9ba625b..0f33a1e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/subquery/in_as_or_4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/subquery/in_as_or_4.plan
@@ -1,28 +1,26 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$21]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- STREAM_SELECT  |LOCAL|
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$22][$#1]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$21]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- STREAM_SELECT  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$22][$#1]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                          -- ASSIGN  |UNPARTITIONED|
-                            -- UNNEST  |UNPARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                        -- ASSIGN  |UNPARTITIONED|
+                          -- UNNEST  |UNPARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/subquery/in_as_or_4_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/subquery/in_as_or_4_ps.plan
index 148d1b0..55d6709 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/subquery/in_as_or_4_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/subquery/in_as_or_4_ps.plan
@@ -6,32 +6,30 @@
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- REPLICATE  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$21]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- STREAM_SELECT  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$22][$#1]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$21]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- STREAM_SELECT  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$22][$#1]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |UNPARTITIONED|
-                                        -- UNNEST  |UNPARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                                    -- ASSIGN  |UNPARTITIONED|
+                                      -- UNNEST  |UNPARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
           -- BROADCAST_EXCHANGE  |PARTITIONED|
             -- AGGREGATE  |UNPARTITIONED|
               -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
@@ -39,29 +37,27 @@
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- REPLICATE  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- STREAM_SELECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_CLUSTERED_GROUP_BY[$$21]  |PARTITIONED|
-                                      {
-                                        -- AGGREGATE  |LOCAL|
-                                          -- STREAM_SELECT  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                      }
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- HYBRID_HASH_JOIN [$$22][$#1]  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_CLUSTERED_GROUP_BY[$$21]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$22][$#1]  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |UNPARTITIONED|
-                                                -- UNNEST  |UNPARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                                            -- ASSIGN  |UNPARTITIONED|
+                                              -- UNNEST  |UNPARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/subquery/in_as_or_6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/subquery/in_as_or_6.plan
index d6e9fa9..ed5067c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/subquery/in_as_or_6.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/subquery/in_as_or_6.plan
@@ -1,28 +1,26 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- SORT_MERGE_EXCHANGE [$$26(ASC) ]  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$22]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- STREAM_SELECT  |LOCAL|
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$23][$#1]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$22]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- STREAM_SELECT  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$23][$#1]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$23]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                          -- ASSIGN  |UNPARTITIONED|
-                            -- UNNEST  |UNPARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                        -- ASSIGN  |UNPARTITIONED|
+                          -- UNNEST  |UNPARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/subquery/in_as_or_6_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/subquery/in_as_or_6_ps.plan
index de7e790..d587010 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/subquery/in_as_or_6_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/subquery/in_as_or_6_ps.plan
@@ -6,32 +6,30 @@
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- REPLICATE  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$22]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- STREAM_SELECT  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$23][$#1]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$22]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- STREAM_SELECT  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$23][$#1]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$23]  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                      -- ASSIGN  |UNPARTITIONED|
-                                        -- UNNEST  |UNPARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                                    -- ASSIGN  |UNPARTITIONED|
+                                      -- UNNEST  |UNPARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
           -- BROADCAST_EXCHANGE  |PARTITIONED|
             -- AGGREGATE  |UNPARTITIONED|
               -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
@@ -39,29 +37,27 @@
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- REPLICATE  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- STREAM_SELECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_CLUSTERED_GROUP_BY[$$22]  |PARTITIONED|
-                                      {
-                                        -- AGGREGATE  |LOCAL|
-                                          -- STREAM_SELECT  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                      }
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- HYBRID_HASH_JOIN [$$23][$#1]  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                -- ASSIGN  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_CLUSTERED_GROUP_BY[$$22]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$23][$#1]  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$23]  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- DATASOURCE_SCAN (tpch.Customer)  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |UNPARTITIONED|
-                                                -- UNNEST  |UNPARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                                            -- ASSIGN  |UNPARTITIONED|
+                                              -- UNNEST  |UNPARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
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 47ede28..aaac258 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
@@ -1,73 +1,69 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     -- STREAM_LIMIT  |UNPARTITIONED|
-      -- STREAM_PROJECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$$136(ASC) ]  |PARTITIONED|
-            -- STREAM_LIMIT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |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[$$145]  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$136(ASC) ]  |PARTITIONED|
+          -- STREAM_LIMIT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [topK: 100] [$$136(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$145]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- HASH_PARTITION_EXCHANGE [$$145]  |PARTITIONED|
+                          -- SORT_GROUP_BY[$$126]  |PARTITIONED|
                                   {
                                     -- AGGREGATE  |LOCAL|
                                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                                   }
-                            -- HASH_PARTITION_EXCHANGE [$$145]  |PARTITIONED|
-                              -- SORT_GROUP_BY[$$126]  |PARTITIONED|
-                                      {
-                                        -- AGGREGATE  |LOCAL|
-                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                      }
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$129][$$132]  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- HYBRID_HASH_JOIN [$$129][$$132]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$137][$$131]  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$137][$$131]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- HYBRID_HASH_JOIN [$$128][$$142]  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- HYBRID_HASH_JOIN [$$128][$$142]  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |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 [$$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|
-                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- HYBRID_HASH_JOIN [$$127][$$140]  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$127]  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (tpcds.store_sales)  |PARTITIONED|
+                                                                  -- DATASOURCE_SCAN (tpcds.customer_address)  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                       -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (tpcds.date_dim)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN (tpcds.item)  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$140]  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- DATASOURCE_SCAN (tpcds.customer)  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- DATASOURCE_SCAN (tpcds.store_sales)  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (tpcds.date_dim)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |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_cbo/tpcds/query-ASTERIXDB-1581.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1581.plan
index 2913fce..6b377c1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1581.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1581.plan
@@ -9,46 +9,146 @@
                 -- ASSIGN  |PARTITIONED|
                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
       -- BROADCAST_EXCHANGE  |LOCAL|
-        -- STREAM_PROJECT  |LOCAL|
-          -- ASSIGN  |LOCAL|
-            -- STREAM_PROJECT  |LOCAL|
-              -- UNNEST  |LOCAL|
-                -- STREAM_PROJECT  |LOCAL|
-                  -- ASSIGN  |LOCAL|
-                    -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                      -- PRE_CLUSTERED_GROUP_BY[$$173]  |LOCAL|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- AGGREGATE  |LOCAL|
-                                    -- STREAM_SELECT  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                          -- STREAM_PROJECT  |LOCAL|
-                            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                              -- HYBRID_HASH_JOIN [$$173][$$174]  |LOCAL|
-                                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$109]  |LOCAL|
-                                          {
-                                            -- AGGREGATE  |LOCAL|
-                                              -- AGGREGATE  |LOCAL|
-                                                -- STREAM_SELECT  |LOCAL|
-                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                          }
-                                    -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                                      -- STABLE_SORT [$$109(ASC)]  |LOCAL|
-                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                          -- STREAM_PROJECT  |UNPARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$109][$$166]  |UNPARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                  -- STREAM_PROJECT  |UNPARTITIONED|
+        -- ASSIGN  |LOCAL|
+          -- UNNEST  |LOCAL|
+            -- ASSIGN  |LOCAL|
+              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                -- PRE_CLUSTERED_GROUP_BY[$$173]  |LOCAL|
+                        {
+                          -- AGGREGATE  |LOCAL|
+                            -- AGGREGATE  |LOCAL|
+                              -- STREAM_SELECT  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                        }
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STREAM_PROJECT  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                        -- HYBRID_HASH_JOIN [$$173][$$174]  |LOCAL|
+                          -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                            -- PRE_CLUSTERED_GROUP_BY[$$109]  |LOCAL|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                                -- STABLE_SORT [$$109(ASC)]  |LOCAL|
+                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                    -- STREAM_PROJECT  |UNPARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$109][$$166]  |UNPARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                            -- ASSIGN  |UNPARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                -- REPLICATE  |UNPARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
                                                     -- ASSIGN  |UNPARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
                                                         -- REPLICATE  |UNPARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                            -- STREAM_PROJECT  |UNPARTITIONED|
-                                                              -- ASSIGN  |UNPARTITIONED|
+                                                            -- AGGREGATE  |UNPARTITIONED|
+                                                              -- AGGREGATE  |UNPARTITIONED|
+                                                                -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                                  -- AGGREGATE  |PARTITIONED|
+                                                                    -- STREAM_SELECT  |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 (tpcds.store_sales)  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                            -- NESTED_LOOP  |UNPARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                -- ASSIGN  |UNPARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                    -- REPLICATE  |UNPARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                        -- ASSIGN  |UNPARTITIONED|
+                                                          -- STREAM_SELECT  |UNPARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                              -- REPLICATE  |UNPARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                                  -- AGGREGATE  |UNPARTITIONED|
+                                                                    -- AGGREGATE  |UNPARTITIONED|
+                                                                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                                        -- AGGREGATE  |PARTITIONED|
+                                                                          -- STREAM_SELECT  |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 (tpcds.store_sales)  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                -- ASSIGN  |UNPARTITIONED|
+                                                  -- AGGREGATE  |UNPARTITIONED|
+                                                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                      -- AGGREGATE  |PARTITIONED|
+                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                          -- ASSIGN  |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 (tpcds.store_sales)  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                            -- NESTED_LOOP  |LOCAL|
+                              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                                -- ASSIGN  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- ASSIGN  |LOCAL|
+                                      -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                                        -- PRE_CLUSTERED_GROUP_BY[$$175]  |LOCAL|
+                                                {
+                                                  -- AGGREGATE  |LOCAL|
+                                                    -- AGGREGATE  |LOCAL|
+                                                      -- STREAM_SELECT  |LOCAL|
+                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                }
+                                          -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                                            -- STABLE_SORT [$$175(ASC)]  |LOCAL|
+                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                -- STREAM_PROJECT  |UNPARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$175][$$176]  |UNPARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                        -- REPLICATE  |UNPARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                            -- ASSIGN  |UNPARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                                -- REPLICATE  |UNPARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                                    -- AGGREGATE  |UNPARTITIONED|
+                                                                      -- AGGREGATE  |UNPARTITIONED|
+                                                                        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                                          -- AGGREGATE  |PARTITIONED|
+                                                                            -- STREAM_SELECT  |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 (tpcds.store_sales)  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                        -- REPLICATE  |UNPARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                            -- ASSIGN  |UNPARTITIONED|
+                                                              -- STREAM_SELECT  |UNPARTITIONED|
                                                                 -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
                                                                   -- REPLICATE  |UNPARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
@@ -57,148 +157,28 @@
                                                                           -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
                                                                             -- AGGREGATE  |PARTITIONED|
                                                                               -- STREAM_SELECT  |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 (tpcds.store_sales)  |PARTITIONED|
-                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                  -- NESTED_LOOP  |UNPARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                      -- STREAM_PROJECT  |UNPARTITIONED|
-                                                        -- ASSIGN  |UNPARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                            -- REPLICATE  |UNPARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                                -- ASSIGN  |UNPARTITIONED|
-                                                                  -- STREAM_SELECT  |UNPARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                                      -- REPLICATE  |UNPARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                                          -- AGGREGATE  |UNPARTITIONED|
-                                                                            -- AGGREGATE  |UNPARTITIONED|
-                                                                              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                                                -- AGGREGATE  |PARTITIONED|
-                                                                                  -- STREAM_SELECT  |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 (tpcds.store_sales)  |PARTITIONED|
-                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                      -- STREAM_PROJECT  |UNPARTITIONED|
-                                                        -- ASSIGN  |UNPARTITIONED|
-                                                          -- AGGREGATE  |UNPARTITIONED|
-                                                            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                              -- AGGREGATE  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- STREAM_SELECT  |PARTITIONED|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                      -- ASSIGN  |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 (tpcds.store_sales)  |PARTITIONED|
-                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                                  -- NESTED_LOOP  |LOCAL|
-                                    -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                                      -- ASSIGN  |LOCAL|
-                                        -- STREAM_PROJECT  |LOCAL|
-                                          -- STREAM_SELECT  |LOCAL|
-                                            -- ASSIGN  |LOCAL|
-                                              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                                                -- PRE_CLUSTERED_GROUP_BY[$$175]  |LOCAL|
-                                                        {
-                                                          -- AGGREGATE  |LOCAL|
-                                                            -- AGGREGATE  |LOCAL|
-                                                              -- STREAM_SELECT  |LOCAL|
-                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                        }
-                                                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                                                    -- STABLE_SORT [$$175(ASC)]  |LOCAL|
-                                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                        -- STREAM_PROJECT  |UNPARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                            -- HYBRID_HASH_JOIN [$$175][$$176]  |UNPARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                                -- REPLICATE  |UNPARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                                    -- STREAM_PROJECT  |UNPARTITIONED|
-                                                                      -- ASSIGN  |UNPARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                                          -- REPLICATE  |UNPARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                                              -- AGGREGATE  |UNPARTITIONED|
-                                                                                -- AGGREGATE  |UNPARTITIONED|
-                                                                                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                                                    -- AGGREGATE  |PARTITIONED|
-                                                                                      -- STREAM_SELECT  |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 (tpcds.store_sales)  |PARTITIONED|
-                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                                -- REPLICATE  |UNPARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                                    -- ASSIGN  |UNPARTITIONED|
-                                                                      -- STREAM_SELECT  |UNPARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                                          -- REPLICATE  |UNPARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                                              -- AGGREGATE  |UNPARTITIONED|
-                                                                                -- AGGREGATE  |UNPARTITIONED|
-                                                                                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                                                    -- AGGREGATE  |PARTITIONED|
-                                                                                      -- STREAM_SELECT  |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 (tpcds.store_sales)  |PARTITIONED|
-                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                      -- STREAM_PROJECT  |UNPARTITIONED|
-                                        -- ASSIGN  |UNPARTITIONED|
-                                          -- AGGREGATE  |UNPARTITIONED|
-                                            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                              -- AGGREGATE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- DATASOURCE_SCAN (tpcds.store_sales)  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                -- ASSIGN  |UNPARTITIONED|
+                                  -- AGGREGATE  |UNPARTITIONED|
+                                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                      -- AGGREGATE  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ASSIGN  |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|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- DATASOURCE_SCAN (tpcds.store_sales)  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |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_cbo/tpcds/query-ASTERIXDB-1591.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1591.plan
index cb5a46e..b9d7b95 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
@@ -1,151 +1,146 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     -- STREAM_LIMIT  |UNPARTITIONED|
-      -- STREAM_PROJECT  |PARTITIONED|
-        -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- SORT_MERGE_EXCHANGE [$$192(ASC) ]  |PARTITIONED|
-              -- STREAM_LIMIT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$192(ASC) ]  |PARTITIONED|
+            -- STREAM_LIMIT  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- SORT_GROUP_BY[$$201, $$202]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- SORT_GROUP_BY[$$201, $$202]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- HASH_PARTITION_EXCHANGE [$$201, $$202]  |PARTITIONED|
+                        -- PRE_CLUSTERED_GROUP_BY[$$189, $$190]  |PARTITIONED|
                                 {
                                   -- AGGREGATE  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
                                 }
-                          -- 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 [$$189(ASC), $$190(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$189(ASC), $$190(ASC)]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- HYBRID_HASH_JOIN [$$189][$$171]  |PARTITIONED|
-                                          -- HASH_PARTITION_EXCHANGE [$$189]  |PARTITIONED|
-                                            -- SORT_GROUP_BY[$$198, $$199]  |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 [$$198, $$199]  |PARTITIONED|
+                                            -- PRE_CLUSTERED_GROUP_BY[$$186, $$187]  |PARTITIONED|
                                                     {
                                                       -- AGGREGATE  |LOCAL|
-                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                        -- STREAM_SELECT  |LOCAL|
+                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                     }
-                                              -- 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 [$$186(ASC), $$187(ASC)]  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STABLE_SORT [$$186(ASC), $$187(ASC)]  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |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[$$195, $$196]  |PARTITIONED|
-                                                                              {
-                                                                                -- AGGREGATE  |LOCAL|
+                                                        -- HYBRID_HASH_JOIN [$$186][$$169]  |PARTITIONED|
+                                                          -- HASH_PARTITION_EXCHANGE [$$186]  |PARTITIONED|
+                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- SORT_GROUP_BY[$$195, $$196]  |PARTITIONED|
+                                                                        {
+                                                                          -- AGGREGATE  |LOCAL|
+                                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                        }
+                                                                  -- HASH_PARTITION_EXCHANGE [$$195, $$196]  |PARTITIONED|
+                                                                    -- PRE_CLUSTERED_GROUP_BY[$$150, $$151]  |PARTITIONED|
+                                                                            {
+                                                                              -- AGGREGATE  |LOCAL|
+                                                                                -- STREAM_SELECT  |LOCAL|
                                                                                   -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                              }
-                                                                        -- 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 [$$150(ASC), $$151(ASC)]  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- HYBRID_HASH_JOIN [$$161][$$151]  |PARTITIONED|
-                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                              -- HYBRID_HASH_JOIN [$$150][$$167]  |PARTITIONED|
-                                                                                                -- HASH_PARTITION_EXCHANGE [$$150]  |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 [$$167]  |PARTITIONED|
-                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                        -- HYBRID_HASH_JOIN [$$172][$$154]  |PARTITIONED|
-                                                                                                          -- HASH_PARTITION_EXCHANGE [$$172]  |PARTITIONED|
-                                                                                                            -- ASSIGN  |PARTITIONED|
-                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                -- ONE_TO_ONE_EXCHANGE  |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 [$$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|
-                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                          -- DATASOURCE_SCAN (tpcds.customer_address)  |PARTITIONED|
-                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                              -- HASH_PARTITION_EXCHANGE [$$169]  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- HYBRID_HASH_JOIN [$$174][$$157]  |PARTITIONED|
-                                                                        -- HASH_PARTITION_EXCHANGE [$$174]  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            }
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- STABLE_SORT [$$150(ASC), $$151(ASC)]  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                             -- STREAM_PROJECT  |PARTITIONED|
                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- DATASOURCE_SCAN (tpcds.web_sales)  |PARTITIONED|
+                                                                                -- HYBRID_HASH_JOIN [$$161][$$151]  |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 [$$154]  |PARTITIONED|
-                                                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                                                    -- STREAM_PROJECT  |PARTITIONED|
                                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- DATASOURCE_SCAN (tpcds.date_dim)  |PARTITIONED|
-                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                          -- HASH_PARTITION_EXCHANGE [$$171]  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                        -- HYBRID_HASH_JOIN [$$150][$$167]  |PARTITIONED|
+                                                                                          -- HASH_PARTITION_EXCHANGE [$$150]  |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 [$$167]  |PARTITIONED|
+                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- HYBRID_HASH_JOIN [$$172][$$154]  |PARTITIONED|
+                                                                                                    -- HASH_PARTITION_EXCHANGE [$$172]  |PARTITIONED|
+                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |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 [$$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|
+                                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                    -- DATASOURCE_SCAN (tpcds.customer_address)  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- HASH_PARTITION_EXCHANGE [$$169]  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- HYBRID_HASH_JOIN [$$174][$$157]  |PARTITIONED|
+                                                                    -- HASH_PARTITION_EXCHANGE [$$174]  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- DATASOURCE_SCAN (tpcds.web_sales)  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- REPLICATE  |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 [$$171]  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- HYBRID_HASH_JOIN [$$176][$$160]  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$176]  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN (tpcds.catalog_sales)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- HYBRID_HASH_JOIN [$$176][$$160]  |PARTITIONED|
-                                                    -- HASH_PARTITION_EXCHANGE [$$176]  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- DATASOURCE_SCAN (tpcds.catalog_sales)  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- REPLICATE  |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|
+                                                      -- REPLICATE  |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|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1596.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1596.plan
index 6b55682..286620f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1596.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1596.plan
@@ -1,27 +1,26 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |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 [$$53][$$57]  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
-                        -- STREAM_PROJECT  |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|
-                              -- HASH_PARTITION_EXCHANGE [$$56, $$55]  |PARTITIONED|
-                                -- DATASOURCE_SCAN (tpcds.catalog_returns)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$57]  |PARTITIONED|
-                        -- DATASOURCE_SCAN (tpcds.item)  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |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 [$$53][$$57]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                      -- STREAM_PROJECT  |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|
+                            -- HASH_PARTITION_EXCHANGE [$$56, $$55]  |PARTITIONED|
+                              -- DATASOURCE_SCAN (tpcds.catalog_returns)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |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_cbo/tpcds/query-ASTERIXDB-1596_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1596_ps.plan
index ea90224..4cc1702 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1596_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1596_ps.plan
@@ -1,58 +1,57 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |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|
+    -- ASSIGN  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |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 [$$53][$$57]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                              -- STREAM_PROJECT  |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|
+                                    -- HASH_PARTITION_EXCHANGE [$$56, $$55]  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (tpcds.catalog_returns)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$57]  |PARTITIONED|
+                              -- DATASOURCE_SCAN (tpcds.item)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$53][$$57]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STREAM_PROJECT  |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|
+                                    -- HYBRID_HASH_JOIN [$$53][$$57]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |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|
+                                              -- HASH_PARTITION_EXCHANGE [$$56, $$55]  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (tpcds.catalog_returns)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$57]  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (tpcds.item)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$56, $$55]  |PARTITIONED|
-                                        -- DATASOURCE_SCAN (tpcds.catalog_returns)  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$57]  |PARTITIONED|
-                                -- DATASOURCE_SCAN (tpcds.item)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                    -- AGGREGATE  |UNPARTITIONED|
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        -- AGGREGATE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- HYBRID_HASH_JOIN [$$53][$$57]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |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|
-                                                -- HASH_PARTITION_EXCHANGE [$$56, $$55]  |PARTITIONED|
-                                                  -- DATASOURCE_SCAN (tpcds.catalog_returns)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |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_cbo/tpch/q12_shipping.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping.plan
index 2b4179a..9308292 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
@@ -1,43 +1,40 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$l_shipmode(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$135]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$118]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$126][$$122]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$126]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- HYBRID_HASH_JOIN [$$118][$$124]  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- STREAM_SELECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$l_shipmode(ASC) ]  |PARTITIONED|
+        -- SORT_GROUP_BY[$$135]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$118]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$118][$$124]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$126][$$122]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$126]  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                  -- UNNEST  |UNPARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$122]  |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|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- UNNEST  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_broadcast.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_broadcast.plan
index 27e4f5f..3a0c212 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_broadcast.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_broadcast.plan
@@ -1,43 +1,40 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$l_shipmode(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$135]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$118]  |PARTITIONED|
-                      {
-                        -- AGGREGATE  |LOCAL|
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                      }
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$125][$$122]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- HYBRID_HASH_JOIN [$$118][$$124]  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- STREAM_SELECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                  -- UNNEST  |UNPARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
+    -- ASSIGN  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$l_shipmode(ASC) ]  |PARTITIONED|
+        -- SORT_GROUP_BY[$$135]  |PARTITIONED|
+                {
+                  -- AGGREGATE  |LOCAL|
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                }
+          -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$118]  |PARTITIONED|
+                    {
+                      -- AGGREGATE  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                    }
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$118][$$124]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$125][$$122]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |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|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- UNNEST  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_broadcast_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_broadcast_ps.plan
index daf42e8..cacd6c1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_broadcast_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_broadcast_ps.plan
@@ -1,95 +1,90 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$l_shipmode(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$l_shipmode(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- SORT_GROUP_BY[$$135]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
-                          -- SORT_GROUP_BY[$$118]  |PARTITIONED|
-                                  {
-                                    -- AGGREGATE  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                  }
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$l_shipmode(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$l_shipmode(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- SORT_GROUP_BY[$$135]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$118]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$118][$$124]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$125][$$122]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- UNNEST  |UNPARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$125][$$122]  |PARTITIONED|
+                              -- SORT_GROUP_BY[$$135]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
+                                  -- SORT_GROUP_BY[$$118]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- HYBRID_HASH_JOIN [$$118][$$124]  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                               -- STREAM_PROJECT  |PARTITIONED|
-                                                -- STREAM_SELECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- HYBRID_HASH_JOIN [$$125][$$122]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                                             -- BROADCAST_EXCHANGE  |PARTITIONED|
                                               -- UNNEST  |UNPARTITIONED|
                                                 -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- SORT_GROUP_BY[$$135]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
-                                    -- SORT_GROUP_BY[$$118]  |PARTITIONED|
-                                            {
-                                              -- AGGREGATE  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                            }
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- HYBRID_HASH_JOIN [$$125][$$122]  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- HYBRID_HASH_JOIN [$$118][$$124]  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- STREAM_SELECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                        -- UNNEST  |UNPARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |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/tpch/q12_shipping_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_ps.plan
index 0d00f7b..1ca832e 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
@@ -1,95 +1,90 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$l_shipmode(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$l_shipmode(ASC)]  |PARTITIONED|
-              -- FORWARD  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- REPLICATE  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- SORT_GROUP_BY[$$135]  |PARTITIONED|
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
-                          -- SORT_GROUP_BY[$$118]  |PARTITIONED|
-                                  {
-                                    -- AGGREGATE  |LOCAL|
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                  }
+    -- ASSIGN  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$l_shipmode(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$l_shipmode(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- SORT_GROUP_BY[$$135]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$118]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$118][$$124]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$126][$$122]  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$126]  |PARTITIONED|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- UNNEST  |UNPARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$126][$$122]  |PARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$126]  |PARTITIONED|
+                              -- SORT_GROUP_BY[$$135]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
+                                  -- SORT_GROUP_BY[$$118]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- HYBRID_HASH_JOIN [$$118][$$124]  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                               -- STREAM_PROJECT  |PARTITIONED|
-                                                -- STREAM_SELECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- HYBRID_HASH_JOIN [$$126][$$122]  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$126]  |PARTITIONED|
+                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                                             -- BROADCAST_EXCHANGE  |PARTITIONED|
                                               -- UNNEST  |UNPARTITIONED|
                                                 -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                    -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  -- AGGREGATE  |UNPARTITIONED|
-                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                      -- AGGREGATE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- REPLICATE  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- SORT_GROUP_BY[$$135]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
-                                    -- SORT_GROUP_BY[$$118]  |PARTITIONED|
-                                            {
-                                              -- AGGREGATE  |LOCAL|
-                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                            }
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- HYBRID_HASH_JOIN [$$126][$$122]  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$126]  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- HYBRID_HASH_JOIN [$$118][$$124]  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- STREAM_SELECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                        -- UNNEST  |UNPARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/non_cloud_sqlpp_queries.xml b/asterixdb/asterix-app/src/test/resources/runtimets/non_cloud_sqlpp_queries.xml
new file mode 100644
index 0000000..f379c0b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/non_cloud_sqlpp_queries.xml
@@ -0,0 +1,93 @@
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements.  See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership.  The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License.  You may obtain a copy of the License at
+ !
+ !   http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied.  See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp"
+            QueryFileExtension=".sqlpp">
+  <test-group name="sqlpp_queries">
+    <test-group name="storage-size">
+      <test-case FilePath="storage-size/datasource-function">
+        <compilation-unit name="collection-exists">
+          <output-dir compare="Clean-JSON">collection-exists</output-dir>
+          <source-location>false</source-location>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="storage-size/datasource-function">
+        <compilation-unit name="non-constant-argument">
+          <output-dir compare="Clean-JSON">non-constant-argument</output-dir>
+          <source-location>false</source-location>
+          <expected-error>Compilation error: Function asterix.storage-size expects constant arguments while arg[0] is of type VARIABLE</expected-error>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="storage-size/datasource-function">
+        <compilation-unit name="collection-does-not-exist">
+          <output-dir compare="Clean-JSON">collection-does-not-exist</output-dir>
+          <source-location>false</source-location>
+          <expected-error>Cannot find dataset with name doesNotExistCollection in dataverse testScope</expected-error>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="storage-size/datasource-function">
+        <compilation-unit name="non-string-argument">
+          <output-dir compare="Clean-JSON">non-string-argument</output-dir>
+          <source-location>false</source-location>
+          <expected-error>Type mismatch: function storage-size expects its 1st input parameter to be of type string, but the actual input type is bigint</expected-error>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="storage-size/datasource-function">
+        <compilation-unit name="view-not-collection">
+          <output-dir compare="Clean-JSON">view-not-collection</output-dir>
+          <source-location>false</source-location>
+          <expected-error>Cannot find dataset with name testCollection_vw in dataverse testScope</expected-error>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="storage-size/datasource-function">
+        <compilation-unit name="external-collection">
+          <output-dir compare="Clean-JSON">external-collection</output-dir>
+          <source-location>false</source-location>
+          <expected-error>Retrieving storage size is not applicable to type: EXTERNAL.</expected-error>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="storage-size/datasource-function">
+        <compilation-unit name="index-exists">
+          <output-dir compare="Clean-JSON">index-exists</output-dir>
+          <source-location>false</source-location>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="storage-size/datasource-function">
+        <compilation-unit name="index-does-not-exist">
+          <output-dir compare="Clean-JSON">index-does-not-exist</output-dir>
+          <source-location>false</source-location>
+          <expected-error>Cannot find index with name testCollection_idx_fake</expected-error>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="storage-size/datasource-function">
+        <compilation-unit name="invalid-arguments-count">
+          <output-dir compare="Clean-JSON">invalid-arguments-count</output-dir>
+          <source-location>false</source-location>
+          <expected-error>Invalid number of arguments for function storage-size</expected-error>
+          <expected-error>Invalid number of arguments for function storage-size</expected-error>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="storage-size/datasource-function">
+        <compilation-unit name="sanity">
+          <output-dir compare="Clean-JSON">sanity</output-dir>
+          <source-location>false</source-location>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+  </test-group>
+</test-suite>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_01/countn_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_01/countn_01.1.ddl.sqlpp
new file mode 100644
index 0000000..ad97753
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_01/countn_01.1.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_01/countn_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_01/countn_01.2.update.sqlpp
new file mode 100644
index 0000000..bd244d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_01/countn_01.2.update.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_01/countn_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_01/countn_01.3.query.sqlpp
new file mode 100644
index 0000000..f19b678
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_01/countn_01.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+
+select element test.array_countn((
+    select element x
+    from  [1,2,3,"four"] as x
+));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_02/countn_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_02/countn_02.1.query.sqlpp
new file mode 100644
index 0000000..261dfa0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_02/countn_02.1.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+select value array_countn(x) from [
+{"x":[1,2,3]},
+{"x":[1,2,null]},
+{"x":[1,"two"]},
+{"x":[1,"two", null]},
+{"x":[null]},
+{"x":[]},
+{"a":[1,2,3]}
+] as list
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_distinct/countn_distinct.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_distinct/countn_distinct.1.query.sqlpp
new file mode 100644
index 0000000..526c423
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_distinct/countn_distinct.1.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ {
+   't1': array_countn(distinct [10,20,30,10,20,30,10]),
+   't2': array_countn(distinct [null,missing,null,missing]),
+   't3': array_countn(distinct [40,null,missing,50,40,null,missing,50,40]),
+   't4': array_countn(distinct [10,20,30,10,20,"thirty",20]),
+   't5': array_countn(distinct [])
+ }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_empty_01/countn_empty_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_empty_01/countn_empty_01.1.ddl.sqlpp
new file mode 100644
index 0000000..0dcc111
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_empty_01/countn_empty_01.1.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests that array_countn aggregation correctly returns 0 for an empty stream,
+ *                  without an aggregate combiner.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_empty_01/countn_empty_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_empty_01/countn_empty_01.2.update.sqlpp
new file mode 100644
index 0000000..de14e51
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_empty_01/countn_empty_01.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests that array_countn aggregation correctly returns 0 for an empty stream,
+ *                  without an aggregate combiner.
+ * Success        : Yes
+ */
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_empty_01/countn_empty_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_empty_01/countn_empty_01.3.query.sqlpp
new file mode 100644
index 0000000..3eec594
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_empty_01/countn_empty_01.3.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests that array_countn aggregation correctly returns null for an empty stream,
+ *                  without an aggregate combiner.
+ * Success        : Yes
+ */
+
+select element array_countn((
+    select element x
+    from  [1,2,3] as x
+    where (x > 10)
+));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_empty_02/countn_empty_02.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_empty_02/countn_empty_02.1.ddl.sqlpp
new file mode 100644
index 0000000..29f79ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_empty_02/countn_empty_02.1.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests that array_countn aggregation correctly returns 0 for an empty stream,
+ *                  with an aggregate combiner.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+ closed {
+  id : bigint,
+  val : double
+};
+
+create  dataset Test(TestType) primary key id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_empty_02/countn_empty_02.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_empty_02/countn_empty_02.2.update.sqlpp
new file mode 100644
index 0000000..379d8e8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_empty_02/countn_empty_02.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests that array_countn aggregation correctly returns 0 for an empty stream,
+ *                  with an aggregate combiner.
+ * Success        : Yes
+ */
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_empty_02/countn_empty_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_empty_02/countn_empty_02.3.query.sqlpp
new file mode 100644
index 0000000..c9ae713
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_empty_02/countn_empty_02.3.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests that array_count aggregation correctly returns 0 for an empty stream,
+ *                  with an aggregate combiner.
+ * Success        : Yes
+ */
+
+use test;
+
+
+select element test.array_count((
+    select element x.val
+    from  Test as x
+));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_null/countn_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_null/countn_null.1.ddl.sqlpp
new file mode 100644
index 0000000..6788afc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_null/countn_null.1.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.NumericType as
+{
+  id : bigint,
+  int8Field : tinyint?,
+  int16Field : smallint?,
+  int32Field : integer?,
+  int64Field : bigint?,
+  floatField : float?,
+  doubleField : double?
+};
+
+create external  dataset Numeric(NumericType) using localfs((`path`=`asterix_nc1://data/nontagged/numericData.json`),(`format`=`adm`));
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_null/countn_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_null/countn_null.2.update.sqlpp
new file mode 100644
index 0000000..bd244d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_null/countn_null.2.update.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_null/countn_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_null/countn_null.3.query.sqlpp
new file mode 100644
index 0000000..8a84b02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/countn_null/countn_null.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+
+{'sql-count':test.array_countn((
+    select element x.doubleField
+    from  Numeric as x
+))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn/scalar_countn.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn/scalar_countn.1.ddl.sqlpp
new file mode 100644
index 0000000..6a5a783
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn/scalar_countn.1.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests the scalar version of array_countn without nulls.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn/scalar_countn.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn/scalar_countn.2.update.sqlpp
new file mode 100644
index 0000000..f0dcccb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn/scalar_countn.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests the scalar version of array_countn without nulls.
+ * Success        : Yes
+ */
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn/scalar_countn.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn/scalar_countn.3.query.sqlpp
new file mode 100644
index 0000000..dce1b5d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn/scalar_countn.3.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests the scalar version of array_countn without nulls.
+ * Success        : Yes
+ */
+
+use test;
+
+
+with  i8 as test.array_countn([test.tinyint('1'),test.tinyint('2'),test.tinyint('3')]),
+      i16 as test.array_countn([test.smallint('1'),test.smallint('2'),test.smallint('3')]),
+      i32 as test.array_countn([test.integer('1'),test.integer('2'),test.integer('3')]),
+      i64 as test.array_countn([test.bigint('1'),test.bigint('2'),test.bigint('3')]),
+      f as test.array_countn([test.float('1'),test.float('2'),test.float('3')]),
+      d as test.array_countn([test.double('1'),test.double('2'),test.double('3')]),
+      s as test.array_countn(['a','b','c'])
+select element i
+from  [i8,i16,i32,i64,f,d,s] as i
+;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn_empty/scalar_countn_empty.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn_empty/scalar_countn_empty.1.ddl.sqlpp
new file mode 100644
index 0000000..b924afc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn_empty/scalar_countn_empty.1.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests the scalar version of array_countn with an empty list.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn_empty/scalar_countn_empty.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn_empty/scalar_countn_empty.2.update.sqlpp
new file mode 100644
index 0000000..ee0152f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn_empty/scalar_countn_empty.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests the scalar version of array_countn with an empty list.
+ * Success        : Yes
+ */
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn_empty/scalar_countn_empty.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn_empty/scalar_countn_empty.3.query.sqlpp
new file mode 100644
index 0000000..901dda1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn_empty/scalar_countn_empty.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests the scalar version of array_countn with an empty list.
+ * Success        : Yes
+ */
+
+use test;
+
+
+select element test.array_countn([]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn_null/scalar_countn_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn_null/scalar_countn_null.1.ddl.sqlpp
new file mode 100644
index 0000000..7a939bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn_null/scalar_countn_null.1.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests the scalar version of array_countn with nulls.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn_null/scalar_countn_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn_null/scalar_countn_null.2.update.sqlpp
new file mode 100644
index 0000000..63eec3e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn_null/scalar_countn_null.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests the scalar version of array_countn with nulls.
+ * Success        : Yes
+ */
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn_null/scalar_countn_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn_null/scalar_countn_null.3.query.sqlpp
new file mode 100644
index 0000000..0f2c33a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/scalar_countn_null/scalar_countn_null.3.query.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests the scalar version of array_countn with nulls.
+ * Success        : Yes
+ */
+
+use test;
+
+
+with  i8 as test.array_countn([test.tinyint('1'),test.tinyint('2'),test.tinyint('3'),null]),
+      i16 as test.array_countn([test.smallint('1'),test.smallint('2'),test.smallint('3'),null]),
+      i32 as test.array_countn([test.integer('1'),test.integer('2'),test.integer('3'),null]),
+      i64 as test.array_countn([test.bigint('1'),test.bigint('2'),test.bigint('3'),null]),
+      f as test.array_countn([test.float('1'),test.float('2'),test.float('3'),null]),
+      d as test.array_countn([test.double('1'),test.double('2'),test.double('3'),null]),
+      s as test.array_countn(['a','b','c',null]),
+      na as test.array_countn([null]),
+      n as test.array_countn(null),
+      m as test.array_countn(missing)
+select element i
+from  [i8,i16,i32,i64,f,d,s,na,n,m] as i
+;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_01/countn_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_01/countn_01.1.ddl.sqlpp
new file mode 100644
index 0000000..ad97753
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_01/countn_01.1.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_01/countn_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_01/countn_01.2.update.sqlpp
new file mode 100644
index 0000000..bd244d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_01/countn_01.2.update.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_01/countn_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_01/countn_01.3.query.sqlpp
new file mode 100644
index 0000000..31e7b07
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_01/countn_01.3.query.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select element test.strict_countn((
+    select element x
+    from  [1,2,3,"four"] as x
+));
+
+select element test.countn(x) from [
+{"x":1},{"x":2},{"x":3},{"x":"four"},{"x":null},{"x":[1,2,3]},{"x":"1"},{"x":{"k":1}}
+] as list;
+
+select n, test.countn(x) from [
+  {"x":1,"n":"a"},
+  {"x":2,"n":"a"},
+  {"x":3,"n":"b"},
+  {"x":"four","n":"a"},
+  {"x":null,"n":"b"},
+  {"x":[1,2,3],"n":"a"},
+  {"x":"1","n":"b"},
+  {"x":{"k":1},"n":"b"},
+  {"n":"b"}
+] as list group by n;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_02/countn_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_02/countn_02.1.query.sqlpp
new file mode 100644
index 0000000..4a10509
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_02/countn_02.1.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+select value strict_countn(x) from [
+{"x":[1,2,3]},
+{"x":[1,2,null]},
+{"x":[1,"two"]},
+{"x":[1,"two", null]},
+{"x":[null]},
+{"x":[]},
+{"a":[1,2,3]}
+] as list
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_distinct/countn_distinct.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_distinct/countn_distinct.1.query.sqlpp
new file mode 100644
index 0000000..699d182
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_distinct/countn_distinct.1.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ {
+   't1': strict_countn(distinct [10,20,30,10,20,30,10]),
+   't2': strict_countn(distinct [null,missing,null,missing]),
+   't3': strict_countn(distinct [40,null,missing,50,40,null,missing,50,40]),
+   't4': strict_countn(distinct [10,20,30,10,20,"thirty",20]),
+   't5': strict_countn(distinct [])
+ }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_empty_01/countn_empty_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_empty_01/countn_empty_01.1.ddl.sqlpp
new file mode 100644
index 0000000..55cfa23
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_empty_01/countn_empty_01.1.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests that countn aggregation correctly returns 0 for an empty stream,
+ *                  without an aggregate combiner.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_empty_01/countn_empty_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_empty_01/countn_empty_01.2.update.sqlpp
new file mode 100644
index 0000000..dcb559a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_empty_01/countn_empty_01.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests that countn aggregation correctly returns 0 for an empty stream,
+ *                  without an aggregate combiner.
+ * Success        : Yes
+ */
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_empty_01/countn_empty_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_empty_01/countn_empty_01.3.query.sqlpp
new file mode 100644
index 0000000..9228733
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_empty_01/countn_empty_01.3.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests that countn aggregation correctly returns null for an empty stream,
+ *                  without an aggregate combiner.
+ * Success        : Yes
+ */
+
+select element strict_countn((
+    select element x
+    from  [1,2,3] as x
+    where (x > 10)
+));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_empty_02/countn_empty_02.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_empty_02/countn_empty_02.1.ddl.sqlpp
new file mode 100644
index 0000000..215fd1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_empty_02/countn_empty_02.1.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests that countn aggregation correctly returns 0 for an empty stream,
+ *                  with an aggregate combiner.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+ closed {
+  id : integer,
+  val : double
+};
+
+create  dataset Test(TestType) primary key id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_empty_02/countn_empty_02.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_empty_02/countn_empty_02.2.update.sqlpp
new file mode 100644
index 0000000..d8ebd7f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_empty_02/countn_empty_02.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests that countn aggregation correctly returns 0 for an empty stream,
+ *                  with an aggregate combiner.
+ * Success        : Yes
+ */
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_empty_02/countn_empty_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_empty_02/countn_empty_02.3.query.sqlpp
new file mode 100644
index 0000000..2844438
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_empty_02/countn_empty_02.3.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests that countn aggregation correctly returns null for an empty stream,
+ *                  with an aggregate combiner.
+ * Success        : Yes
+ */
+
+use test;
+
+
+select element test.strict_countn((
+    select element x.val
+    from  Test as x
+));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_null/countn_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_null/countn_null.1.ddl.sqlpp
new file mode 100644
index 0000000..3f2c5bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_null/countn_null.1.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.NumericType as
+{
+  id : integer,
+  int8Field : tinyint?,
+  int16Field : smallint?,
+  int32Field : integer?,
+  int64Field : bigint?,
+  floatField : float?,
+  doubleField : double?
+};
+
+create external  dataset Numeric(NumericType) using localfs((`path`=`asterix_nc1://data/nontagged/numericData.json`),(`format`=`adm`));
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_null/countn_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_null/countn_null.2.update.sqlpp
new file mode 100644
index 0000000..bd244d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_null/countn_null.2.update.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_null/countn_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_null/countn_null.3.query.sqlpp
new file mode 100644
index 0000000..d5a60c4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/countn_null/countn_null.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+
+{'count':test.strict_countn((
+    select element x.doubleField
+    from  Numeric as x
+))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn/scalar_countn.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn/scalar_countn.1.ddl.sqlpp
new file mode 100644
index 0000000..1172330
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn/scalar_countn.1.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests the scalar version of count without nulls.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn/scalar_countn.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn/scalar_countn.2.update.sqlpp
new file mode 100644
index 0000000..411b984
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn/scalar_countn.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests the scalar version of count without nulls.
+ * Success        : Yes
+ */
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn/scalar_countn.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn/scalar_countn.3.query.sqlpp
new file mode 100644
index 0000000..dd1b016
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn/scalar_countn.3.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests the scalar version of count without nulls.
+ * Success        : Yes
+ */
+
+use test;
+
+
+with  i8 as test.strict_countn([test.tinyint('1'),test.tinyint('2'),test.tinyint('3')]),
+      i16 as test.strict_countn([test.smallint('1'),test.smallint('2'),test.smallint('3')]),
+      i32 as test.strict_countn([test.integer('1'),test.integer('2'),test.integer('3')]),
+      i64 as test.strict_countn([test.bigint('1'),test.bigint('2'),test.bigint('3')]),
+      f as test.strict_countn([test.float('1'),test.float('2'),test.float('3')]),
+      d as test.strict_countn([test.double('1'),test.double('2'),test.double('3')]),
+      s as test.strict_countn(['a','b','c'])
+select element i
+from  [i8,i16,i32,i64,f,d,s] as i
+;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn_empty/scalar_countn_empty.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn_empty/scalar_countn_empty.1.ddl.sqlpp
new file mode 100644
index 0000000..bbcfff8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn_empty/scalar_countn_empty.1.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests the scalar version of countn with an empty list.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn_empty/scalar_countn_empty.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn_empty/scalar_countn_empty.2.update.sqlpp
new file mode 100644
index 0000000..e6cd404
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn_empty/scalar_countn_empty.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests the scalar version of countn with an empty list.
+ * Success        : Yes
+ */
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn_empty/scalar_countn_empty.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn_empty/scalar_countn_empty.3.query.sqlpp
new file mode 100644
index 0000000..201aef3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn_empty/scalar_countn_empty.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests the scalar version of countn with an empty list.
+ * Success        : Yes
+ */
+
+use test;
+
+
+select element test.strict_countn([]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn_null/scalar_countn_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn_null/scalar_countn_null.1.ddl.sqlpp
new file mode 100644
index 0000000..cb7c70c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn_null/scalar_countn_null.1.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests the scalar version of countn with nulls.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn_null/scalar_countn_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn_null/scalar_countn_null.2.update.sqlpp
new file mode 100644
index 0000000..b7da5af
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn_null/scalar_countn_null.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests the scalar version of countn with nulls.
+ * Success        : Yes
+ */
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn_null/scalar_countn_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn_null/scalar_countn_null.3.query.sqlpp
new file mode 100644
index 0000000..393d16e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_countn_null/scalar_countn_null.3.query.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests the scalar version of countn with nulls.
+ * Success        : Yes
+ */
+
+use test;
+
+
+with  i8 as test.strict_countn([test.tinyint('1'),test.tinyint('2'),test.tinyint('3'),null]),
+      i16 as test.strict_countn([test.smallint('1'),test.smallint('2'),test.smallint('3'),null]),
+      i32 as test.strict_countn([test.integer('1'),test.integer('2'),test.integer('3'),null]),
+      i64 as test.strict_countn([test.bigint('1'),test.bigint('2'),test.bigint('3'),null]),
+      f as test.strict_countn([test.float('1'),test.float('2'),test.float('3'),null]),
+      d as test.strict_countn([test.double('1'),test.double('2'),test.double('3'),null]),
+      s as test.strict_countn(['a','b','c',null]),
+      na as test.strict_countn([null]),
+      n as test.strict_countn(null),
+      m as test.strict_countn(missing)
+select element i
+from  [i8,i16,i32,i64,f,d,s,na,n,m] as i
+;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_bracket_notation/array_slice_bracket_notation.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_bracket_notation/array_slice_bracket_notation.1.ddl.sqlpp
new file mode 100755
index 0000000..8103562
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_bracket_notation/array_slice_bracket_notation.1.ddl.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+CREATE TYPE openType AS {id: int};
+CREATE DATASET ds(openType) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_bracket_notation/array_slice_bracket_notation.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_bracket_notation/array_slice_bracket_notation.2.update.sqlpp
new file mode 100755
index 0000000..a9171ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_bracket_notation/array_slice_bracket_notation.2.update.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+UPSERT INTO ds {"id": 1, "arr": [0, 1, 2, 3, 4, 5]};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_bracket_notation/array_slice_bracket_notation.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_bracket_notation/array_slice_bracket_notation.3.query.sqlpp
new file mode 100755
index 0000000..91a479a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_bracket_notation/array_slice_bracket_notation.3.query.sqlpp
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+with arr as [0, 1, 2, 3, 4, 5] select value
+{
+  "t1": (arr[2]),
+  "t2": (arr[2:4]),
+  "t3": (arr[2:10]),
+  "t4": (arr[-7:4]),
+  "t5": (arr[-7:10]),
+  "t6": (arr[-3:4]),
+  "t7": (arr[2:-1]),
+  "t8": (arr[2:]),
+  "t9": (arr[10:]),
+  "t10": (arr[-7:]),
+  "t11": (arr[:]),
+  "t12": (arr[0]),
+  "t13": (arr[:4]),
+  "t14": (arr[:10]),
+  "t15": (arr[:-7]),
+  "t16": ([][0:]),
+  "t17": ([][:0]),
+  "t18": (select value ds.arr[1:3] from ds),
+  "t19": (select value ds.arr[1:] from ds),
+  "t20": (select value ds.arr[:3] from ds),
+  "t21": (select value ds.arr[] from ds),
+  "t22": (select value ds.arr[:] from ds),
+  "t20": (select value ds.arr[:10] from ds)
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_bracket_notation/array_slice_bracket_notation.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_bracket_notation/array_slice_bracket_notation.4.ddl.sqlpp
new file mode 100755
index 0000000..43604c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_bracket_notation/array_slice_bracket_notation.4.ddl.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATASET ds;
+DROP TYPE openType;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-does-not-exist/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-does-not-exist/test.040.query.sqlpp
new file mode 100644
index 0000000..dc66c29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-does-not-exist/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "doesNotExistCollection") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-does-not-exist/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-does-not-exist/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-does-not-exist/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-exists/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-exists/test.010.ddl.sqlpp
new file mode 100644
index 0000000..a2ccf84
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-exists/test.010.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
+CREATE DATABASE testDatabase;
+CREATE DATAVERSE testDatabase.testScope;
+USE testDatabase.testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-exists/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-exists/test.020.update.sqlpp
new file mode 100644
index 0000000..4383138
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-exists/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testDatabase.testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-exists/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-exists/test.040.query.sqlpp
new file mode 100644
index 0000000..d4442c7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-exists/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "testCollection") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-exists/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-exists/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/collection-exists/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/external-collection/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/external-collection/test.010.ddl.sqlpp
new file mode 100644
index 0000000..5ae3310
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/external-collection/test.010.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
+CREATE DATABASE testDatabase;
+CREATE DATAVERSE testDatabase.testScope;
+USE testDatabase.testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
+
+CREATE TYPE test AS open {};
+CREATE EXTERNAL DATASET testExternalCollection(test) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("sessionToken"="dummySessionToken"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://127.0.0.1:8001"),
+("container"="playground"),
+("definition"="json-data/reviews/single-line/json"),
+("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/external-collection/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/external-collection/test.020.update.sqlpp
new file mode 100644
index 0000000..4383138
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/external-collection/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testDatabase.testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/external-collection/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/external-collection/test.040.query.sqlpp
new file mode 100644
index 0000000..9915cfd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/external-collection/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "testExternalCollection") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/external-collection/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/external-collection/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/external-collection/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.010.ddl.sqlpp
new file mode 100644
index 0000000..a2ccf84
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.010.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
+CREATE DATABASE testDatabase;
+CREATE DATAVERSE testDatabase.testScope;
+USE testDatabase.testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.020.update.sqlpp
new file mode 100644
index 0000000..4383138
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testDatabase.testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.021.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.021.update.sqlpp
new file mode 100644
index 0000000..18ca3d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.021.update.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testDatabase.testScope;
+
+CREATE INDEX testCollection_idx ON testCollection(some_value: int);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.040.query.sqlpp
new file mode 100644
index 0000000..905a20a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "testCollection", "testCollection_idx_fake") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-does-not-exist/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.010.ddl.sqlpp
new file mode 100644
index 0000000..a2ccf84
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.010.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
+CREATE DATABASE testDatabase;
+CREATE DATAVERSE testDatabase.testScope;
+USE testDatabase.testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.020.update.sqlpp
new file mode 100644
index 0000000..4383138
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testDatabase.testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.021.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.021.update.sqlpp
new file mode 100644
index 0000000..18ca3d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.021.update.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testDatabase.testScope;
+
+CREATE INDEX testCollection_idx ON testCollection(some_value: int);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.040.query.sqlpp
new file mode 100644
index 0000000..cb60cb9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "testCollection", "testCollection_idx") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/index-exists/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.010.ddl.sqlpp
new file mode 100644
index 0000000..a2ccf84
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.010.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
+CREATE DATABASE testDatabase;
+CREATE DATAVERSE testDatabase.testScope;
+USE testDatabase.testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.020.update.sqlpp
new file mode 100644
index 0000000..4383138
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testDatabase.testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.040.query.sqlpp
new file mode 100644
index 0000000..d376541
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.041.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.041.query.sqlpp
new file mode 100644
index 0000000..b395ed5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.041.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "testCollection", "testIndex", "tooManyArgs") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/invalid-arguments-count/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-constant-argument/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-constant-argument/test.010.ddl.sqlpp
new file mode 100644
index 0000000..a2ccf84
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-constant-argument/test.010.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
+CREATE DATABASE testDatabase;
+CREATE DATAVERSE testDatabase.testScope;
+USE testDatabase.testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-constant-argument/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-constant-argument/test.020.update.sqlpp
new file mode 100644
index 0000000..4383138
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-constant-argument/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testDatabase.testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-constant-argument/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-constant-argument/test.040.query.sqlpp
new file mode 100644
index 0000000..aea887f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-constant-argument/test.040.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testDatabase.testScope;
+SELECT storage_size(v.id, v.id, v.id) FROM testCollection AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-constant-argument/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-constant-argument/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-constant-argument/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-string-argument/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-string-argument/test.010.ddl.sqlpp
new file mode 100644
index 0000000..a2ccf84
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-string-argument/test.010.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
+CREATE DATABASE testDatabase;
+CREATE DATAVERSE testDatabase.testScope;
+USE testDatabase.testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-string-argument/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-string-argument/test.020.update.sqlpp
new file mode 100644
index 0000000..4383138
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-string-argument/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testDatabase.testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-string-argument/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-string-argument/test.040.query.sqlpp
new file mode 100644
index 0000000..ddb1996
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-string-argument/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT VALUE v FROM storage_size(1, 2, 3) AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-string-argument/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-string-argument/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/non-string-argument/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/sanity/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/sanity/test.010.ddl.sqlpp
new file mode 100644
index 0000000..84d75e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/sanity/test.010.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
+CREATE DATABASE testDatabase;
+CREATE DATAVERSE testDatabase.testScope;
+USE testDatabase.testScope;
+
+CREATE COLLECTION testCollection PRIMARY KEY (id: int);
+
+CREATE INDEX testCollection_idx ON testCollection(someField1: int);
+
+CREATE COLLECTION collectionAndIndexes PRIMARY KEY (id: int);
+CREATE COLLECTION collectionOnly PRIMARY KEY (id: int);
+CREATE COLLECTION indexOnly PRIMARY KEY (id: int)
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/sanity/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/sanity/test.020.update.sqlpp
new file mode 100644
index 0000000..f08857d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/sanity/test.020.update.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testDatabase.testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {
+    "id": intVal,
+    "someField1": intVal,
+    "someField2": uuid(),
+    "someField3": {"firstName": uuid(), "lastName": uuid(), "location": {"lon": intVal * 2 + intVal, "lat": intVal * 7 + intVal}}
+    }
+    FROM range(1, 100000) intVal
+);
+
+INSERT INTO collectionAndIndexes({"id": 1, "valueField": (select value size from storage_size("testDatabase", "testScope", "testCollection") as x)[0]});
+INSERT INTO collectionOnly({"id": 1, "valueField": (select value size from storage_size("testDatabase", "testScope", "testCollection", "testCollection") as x)[0]});
+INSERT INTO indexOnly({"id": 1, "valueField": (select value size from storage_size("testDatabase", "testScope", "testCollection", "testCollection_idx") as x)[0]});
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/sanity/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/sanity/test.040.query.sqlpp
new file mode 100644
index 0000000..f37a396
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/sanity/test.040.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testDatabase.testScope;
+
+SELECT a.valueField > b.valueField AS collectionAndIndexesLargerThanCollectionOnly, b.valueField > c.valueField AS collectionOnlyLargerThanIndexOnly
+FROM collectionAndIndexes a,
+     collectionOnly b,
+     indexOnly c
+WHERE a.id = b.id
+  AND a.id = c.id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/sanity/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/sanity/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/sanity/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/view-not-collection/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/view-not-collection/test.010.ddl.sqlpp
new file mode 100644
index 0000000..3d9ea2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/view-not-collection/test.010.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
+CREATE DATABASE testDatabase;
+CREATE DATAVERSE testDatabase.testScope;
+USE testDatabase.testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
+
+CREATE VIEW testCollection_vw(id int)
+DEFAULT NULL datetime "YYYY-MM-DD hh:mm:ss" AS
+SELECT t.id
+FROM testCollection t;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/view-not-collection/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/view-not-collection/test.020.update.sqlpp
new file mode 100644
index 0000000..4383138
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/view-not-collection/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testDatabase.testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/view-not-collection/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/view-not-collection/test.040.query.sqlpp
new file mode 100644
index 0000000..cd88a1e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/view-not-collection/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "testCollection_vw") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/view-not-collection/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/view-not-collection/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cloud_storage/storage-size/datasource-function/view-not-collection/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-cover-data-types/parquet-cover-data-types.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-cover-data-types/parquet-cover-data-types.01.ddl.sqlpp
new file mode 100644
index 0000000..56e79c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-cover-data-types/parquet-cover-data-types.01.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test if exists;
+CREATE DATAVERSE test;
+USE test;
+
+
+CREATE TYPE ColumnType1 AS {
+  id: integer,
+  name : string
+};
+
+CREATE COLLECTION TestCollection(ColumnType1) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-cover-data-types/parquet-cover-data-types.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-cover-data-types/parquet-cover-data-types.02.update.sqlpp
new file mode 100644
index 0000000..d2a376c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-cover-data-types/parquet-cover-data-types.02.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into TestCollection({"id":18, "name": "Virat" , "dateType":date("1988-11-05"), "timeType": time("03:10:00.493Z") , "boolType" : false , "doubleType" : 0.75, "datetimeType" : datetime("1900-02-01T00:00:00") });
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-cover-data-types/parquet-cover-data-types.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-cover-data-types/parquet-cover-data-types.03.update.sqlpp
new file mode 100644
index 0000000..a95146f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-cover-data-types/parquet-cover-data-types.03.update.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+   select c.* from TestCollection c
+) toWriter
+TO hdfs
+PATH ("copy-to-result", "parquet-cover-data-types")
+TYPE ( {   name : string,  id : int,  dateType : date,   timeType : time,  boolType : boolean,   doubleType : double,  datetimeType : datetime   } )
+WITH {
+    "hdfs":"hdfs://127.0.0.1:31888/",
+    "format":"parquet"
+};
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-cover-data-types/parquet-cover-data-types.04.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-cover-data-types/parquet-cover-data-types.04.ddl.sqlpp
new file mode 100644
index 0000000..f23893b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-cover-data-types/parquet-cover-data-types.04.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+CREATE TYPE ColumnType2 AS {
+};
+
+
+
+CREATE EXTERNAL DATASET TestDataset(ColumnType2) USING hdfs
+(
+  ("hdfs"="hdfs://127.0.0.1:31888/"),
+  ("path"="copy-to-result/parquet-cover-data-types/"),
+  ("include"="*.parquet"),
+  ("input-format"="parquet-input-format"),
+  ("requireVersionChangeDetection"="false"),
+  ("format" = "parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-cover-data-types/parquet-cover-data-types.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-cover-data-types/parquet-cover-data-types.05.query.sqlpp
new file mode 100644
index 0000000..b03fc5e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-cover-data-types/parquet-cover-data-types.05.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT c.*
+FROM TestDataset c
+ORDER BY c.id;
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-empty-array/parquet-empty-array.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-empty-array/parquet-empty-array.01.ddl.sqlpp
new file mode 100644
index 0000000..6be9489
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-empty-array/parquet-empty-array.01.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test if exists;
+CREATE DATAVERSE test;
+USE test;
+
+
+CREATE TYPE ColumnType1 AS {
+    id: integer
+};
+
+CREATE COLLECTION TestCollection(ColumnType1) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-empty-array/parquet-empty-array.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-empty-array/parquet-empty-array.02.update.sqlpp
new file mode 100644
index 0000000..d569ee5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-empty-array/parquet-empty-array.02.update.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+INSERT INTO TestCollection {"id":2,"name":{"first":["power","star"]}};
+INSERT INTO TestCollection {"id":5,"name":{"first":[]}};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-empty-array/parquet-empty-array.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-empty-array/parquet-empty-array.03.update.sqlpp
new file mode 100644
index 0000000..5a66928
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-empty-array/parquet-empty-array.03.update.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+select c.* from TestCollection c
+    ) toWriter
+TO hdfs
+PATH ("copy-to-result", "parquet-empty-array")
+TYPE ( { id : int,  name : { first :  [ string ] } } )
+WITH {
+    "hdfs":"hdfs://127.0.0.1:31888",
+    "format":"parquet"
+    };
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-empty-array/parquet-empty-array.04.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-empty-array/parquet-empty-array.04.ddl.sqlpp
new file mode 100644
index 0000000..a7ee67f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-empty-array/parquet-empty-array.04.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+CREATE TYPE ColumnType2 AS {
+    };
+
+
+
+CREATE EXTERNAL DATASET TestDataset(ColumnType2) USING hdfs
+(
+  ("hdfs"="hdfs://127.0.0.1:31888/"),
+  ("path"="copy-to-result/parquet-empty-array/"),
+  ("include"="*.parquet"),
+  ("input-format"="parquet-input-format"),
+  ("requireVersionChangeDetection"="false"),
+  ("format" = "parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-empty-array/parquet-empty-array.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-empty-array/parquet-empty-array.05.query.sqlpp
new file mode 100644
index 0000000..b03fc5e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-empty-array/parquet-empty-array.05.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT c.*
+FROM TestDataset c
+ORDER BY c.id;
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-simple/parquet-simple.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-simple/parquet-simple.01.ddl.sqlpp
new file mode 100644
index 0000000..76970a5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-simple/parquet-simple.01.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test if exists;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE ColumnType2 AS {
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-simple/parquet-simple.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-simple/parquet-simple.02.update.sqlpp
new file mode 100644
index 0000000..1183752
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-simple/parquet-simple.02.update.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+COPY (
+   select "123" as id
+) toWriter
+TO hdfs
+PATH ("copy-to-result", "parquet-simple")
+TYPE ( {id:string} )
+WITH {
+    "hdfs":"hdfs://127.0.0.1:31888",
+    "format":"parquet",
+    "version" : "2"
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-simple/parquet-simple.03.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-simple/parquet-simple.03.ddl.sqlpp
new file mode 100644
index 0000000..e31c9b0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-simple/parquet-simple.03.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+CREATE EXTERNAL DATASET DatasetCopy(ColumnType2) USING hdfs
+(
+  ("hdfs"="hdfs://127.0.0.1:31888/"),
+  ("path"="copy-to-result/parquet-simple"),
+  ("format" = "parquet"),
+  ("input-format"="parquet-input-format"),
+  ("requireVersionChangeDetection"="false"),
+  ("include"="*.parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-simple/parquet-simple.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-simple/parquet-simple.04.query.sqlpp
new file mode 100644
index 0000000..5aeedb8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-simple/parquet-simple.04.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT id
+FROM DatasetCopy c
+ORDER BY c.id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-tweet/parquet-tweet.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-tweet/parquet-tweet.01.ddl.sqlpp
new file mode 100644
index 0000000..f890e0d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-tweet/parquet-tweet.01.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test if exists;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE ColumnType1 AS {
+    id: string
+};
+
+CREATE DATASET DummyTweetDataset(ColumnType1)
+PRIMARY KEY id WITH {
+    "storage-format": {"format" : "column"}
+};
+
+
+CREATE TYPE ColumnType2 AS {
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-tweet/parquet-tweet.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-tweet/parquet-tweet.02.update.sqlpp
new file mode 100644
index 0000000..83a1140
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-tweet/parquet-tweet.02.update.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+LOAD DATASET DummyTweetDataset USING localfs
+(
+    ("path" = "asterix_nc1://data/hdfs/parquet/dummy_tweet.json"),
+    ("format" = "json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-tweet/parquet-tweet.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-tweet/parquet-tweet.03.update.sqlpp
new file mode 100644
index 0000000..9bae74b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-tweet/parquet-tweet.03.update.sqlpp
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+   SELECT c.* FROM DummyTweetDataset c
+) toWriter
+TO hdfs
+PATH ("copy-to-result", "parquet-tweet")
+TYPE ( {
+               coordinates: {
+                 coordinates: [
+                   double
+                 ],
+                 `type` : string
+               },
+               created_at: string,
+               entities: {
+                 urls: [
+                   {
+                     display_url: string,
+                     expanded_url: string,
+                     indices: [
+                       int
+                     ],
+                     url: string
+                   }
+                 ],
+                 user_mentions: [
+                   {
+                     id: int,
+                     id_str: string,
+                     indices: [
+                       int
+                     ],
+                     name: string,
+                     screen_name: string
+                   }
+                 ]
+               },
+               favorite_count: int,
+               favorited: boolean,
+               filter_level: string,
+               geo: {
+                 coordinates: [
+                   double
+                 ],
+                 `type`: string
+               },
+               id: string,
+               id_str: string,
+               in_reply_to_screen_name: string,
+               in_reply_to_status_id: int,
+               in_reply_to_status_id_str: string,
+               in_reply_to_user_id: int,
+               in_reply_to_user_id_str: string,
+               is_quote_status: boolean,
+               lang: string,
+               place: {
+                 bounding_box: {
+                   coordinates: [
+                     [
+                       [
+                         double
+                       ]
+                     ]
+                   ],
+                   `type`: string
+                 },
+                 country: string,
+                 country_code: string,
+                 full_name: string,
+                 id: string,
+                 name: string,
+                 place_type: string,
+                 url: string
+               },
+               possibly_sensitive: boolean,
+               quoted_status: {
+                 created_at: string,
+                 entities: {
+                   user_mentions: [
+                     {
+                       id: int,
+                       id_str: string,
+                       indices: [
+                         int
+                       ],
+                       name: string,
+                       screen_name: string
+                     }
+                   ]
+                 },
+                 favorite_count: int,
+                 favorited: boolean,
+                 filter_level: string,
+                 id: int,
+                 id_str: string,
+                 in_reply_to_screen_name: string,
+                 in_reply_to_status_id: int,
+                 in_reply_to_status_id_str: string,
+                 in_reply_to_user_id: int,
+                 in_reply_to_user_id_str: string,
+                 is_quote_status: boolean,
+                 lang: string,
+                 retweet_count: int,
+                 retweeted: boolean,
+                 source: string,
+                 text: string,
+                 truncated: boolean,
+                 user: {
+                   contributors_enabled: boolean,
+                   created_at: string,
+                   default_profile: boolean,
+                   default_profile_image: boolean,
+                   description: string,
+                   favourites_count: int,
+                   followers_count: int,
+                   friends_count: int,
+                   geo_enabled: boolean,
+                   id: int,
+                   id_str: string,
+                   is_translator: boolean,
+                   lang: string,
+                   listed_count: int,
+                   name: string,
+                   profile_background_color: string,
+                   profile_background_image_url: string,
+                   profile_background_image_url_https: string,
+                   profile_background_tile: boolean,
+                   profile_banner_url: string,
+                   profile_image_url: string,
+                   profile_image_url_https: string,
+                   profile_link_color: string,
+                   profile_sidebar_border_color: string,
+                   profile_sidebar_fill_color: string,
+                   profile_text_color: string,
+                   profile_use_background_image: boolean,
+                   protected: boolean,
+                   screen_name: string,
+                   statuses_count: int,
+                   verified: boolean
+                 }
+               },
+               quoted_status_id: int,
+               quoted_status_id_str: string,
+               retweet_count: int,
+               retweeted: boolean,
+               source: string,
+               text: string,
+               timestamp_ms: string,
+               truncated: boolean,
+               user: {
+                 contributors_enabled: boolean,
+                 created_at: string,
+                 default_profile: boolean,
+                 default_profile_image: boolean,
+                 description: string,
+                 favourites_count: int,
+                 followers_count: int,
+                 friends_count: int,
+                 geo_enabled: boolean,
+                 id: int,
+                 id_str: string,
+                 is_translator: boolean,
+                 lang: string,
+                 listed_count: int,
+                 location: string,
+                 name: string,
+                 profile_background_color: string,
+                 profile_background_image_url: string,
+                 profile_background_image_url_https: string,
+                 profile_background_tile: boolean,
+                 profile_banner_url: string,
+                 profile_image_url: string,
+                 profile_image_url_https: string,
+                 profile_link_color: string,
+                 profile_sidebar_border_color: string,
+                 profile_sidebar_fill_color: string,
+                 profile_text_color: string,
+                 profile_use_background_image: boolean,
+                 protected: boolean,
+                 screen_name: string,
+                 statuses_count: int,
+                 time_zone: string,
+                 url: string,
+                 utc_offset: int,
+                 verified: boolean
+               }
+             } )
+WITH {
+    "hdfs":"hdfs://127.0.0.1:31888",
+    "format":"parquet"
+};
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-tweet/parquet-tweet.04.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-tweet/parquet-tweet.04.ddl.sqlpp
new file mode 100644
index 0000000..62467d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-tweet/parquet-tweet.04.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+
+
+CREATE EXTERNAL DATASET DummyTweetDatasetCopy(ColumnType2) USING hdfs
+(
+  ("hdfs"="hdfs://127.0.0.1:31888/"),
+  ("path"="copy-to-result/parquet-tweet/"),
+  ("include"="*.parquet"),
+  ("input-format"="parquet-input-format"),
+  ("requireVersionChangeDetection"="false"),
+  ("format" = "parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-tweet/parquet-tweet.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-tweet/parquet-tweet.05.query.sqlpp
new file mode 100644
index 0000000..13587f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-tweet/parquet-tweet.05.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT c.*
+FROM DummyTweetDatasetCopy c
+ORDER BY c.id;
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-utf8/parquet-utf8.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-utf8/parquet-utf8.01.ddl.sqlpp
new file mode 100644
index 0000000..dfc64ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-utf8/parquet-utf8.01.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test if exists;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE ColumnType1 AS {
+    id: int
+};
+
+CREATE DATASET NameCommentDataset(ColumnType1)
+PRIMARY KEY id WITH {
+    "storage-format": {"format" : "column"}
+};
+
+
+CREATE TYPE ColumnType2 AS {
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-utf8/parquet-utf8.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-utf8/parquet-utf8.02.update.sqlpp
new file mode 100644
index 0000000..8591369
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-utf8/parquet-utf8.02.update.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+LOAD DATASET NameCommentDataset USING localfs
+(
+    ("path" = "asterix_nc1://data/hdfs/parquet/id_name_comment.json"),
+    ("format" = "json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-utf8/parquet-utf8.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-utf8/parquet-utf8.03.update.sqlpp
new file mode 100644
index 0000000..79b2dda
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-utf8/parquet-utf8.03.update.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+   SELECT c.* FROM NameCommentDataset c
+) toWriter
+TO hdfs
+PATH ("copy-to-result", "parquet-utf8")
+TYPE ( { comment:string, id:bigint, name:string } )
+WITH {
+    "hdfs":"hdfs://127.0.0.1:31888",
+    "format":"parquet"
+};
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-utf8/parquet-utf8.04.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-utf8/parquet-utf8.04.ddl.sqlpp
new file mode 100644
index 0000000..49c9b0f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-utf8/parquet-utf8.04.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+
+
+CREATE EXTERNAL DATASET NameCommentDatasetCopy(ColumnType2) USING hdfs
+(
+  ("hdfs"="hdfs://127.0.0.1:31888"),
+  ("path"="copy-to-result/parquet-utf8/"),
+  ("include"="*.parquet"),
+  ("input-format"="parquet-input-format"),
+  ("requireVersionChangeDetection"="false"),
+  ("format" = "parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-utf8/parquet-utf8.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-utf8/parquet-utf8.05.query.sqlpp
new file mode 100644
index 0000000..17cd027
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to-hdfs/parquet-utf8/parquet-utf8.05.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT c.*
+FROM NameCommentDatasetCopy c
+ORDER BY c.id;
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.01.ddl.sqlpp
new file mode 100644
index 0000000..80a8c34
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.01.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+CREATE TYPE ColumnType AS {
+  id: bigint,
+  name: string,
+  amount: float,
+  accountNumber: double
+};
+
+CREATE COLLECTION TestCollection(ColumnType) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.02.update.sqlpp
new file mode 100644
index 0000000..d57a311
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.02.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+USE test;
+
+INSERT INTO TestCollection({"id":1, "name":"Macbook1", "amount":123.2, "accountNumber":345.34});
+INSERT INTO TestCollection({"id":2, "name":"Macbook2", "amount":456.7, "accountNumber":123.45});
+INSERT INTO TestCollection({"id":3, "name":"Macbook3", "amount":789.1, "accountNumber":678.90});
+INSERT INTO TestCollection({"id":4, "name":"Macbook4", "amount":234.5, "accountNumber":567.89});
+INSERT INTO TestCollection({"id":5, "name":"Macbook5", "amount":876.5, "accountNumber":345.67});
+INSERT INTO TestCollection({"id":6, "name":"Macbook6", "amount":345.6, "accountNumber":987.65});
+INSERT INTO TestCollection({"id":7, "name":"Macbook7", "amount":678.9, "accountNumber":234.56});
+INSERT INTO TestCollection({"id":8, "name":"Macbook8", "amount":987.2, "accountNumber":789.12});
+INSERT INTO TestCollection({"id":9, "name":"Macbook9", "amount":543.2, "accountNumber":321.45});
+INSERT INTO TestCollection({"id":10, "name":"Macbook10", "amount":123.9, "accountNumber":654.32});
+INSERT INTO TestCollection({"id":11, "name":"Macbook11", "amount":567.8, "accountNumber":456.78});
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.03.update.sqlpp
new file mode 100644
index 0000000..606457a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.03.update.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+USE test;
+
+COPY (
+   SELECT id, name, amount, accountNumber FROM TestCollection
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv", "delimiter")
+AS (id bigint, name STRING, amount float, accountNumber double)
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"csv",
+    "delimiter":"|"
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.04.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.04.ddl.sqlpp
new file mode 100644
index 0000000..2208b2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.04.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+CREATE EXTERNAL DATASET DatasetCopy(ColumnType) USING %adapter%
+(
+  %template%,
+  ("header"="false"),
+  ("delimiter"="|"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%copy-to-result/csv/delimiter"),
+  ("format" = "csv"),
+  ("requireVersionChangeDetection"="false"),
+  ("include"="*.csv")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.05.query.sqlpp
new file mode 100644
index 0000000..b407ec4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.05.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT id, name, amount, accountNumber
+FROM DatasetCopy d order by d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.06.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.06.ddl.sqlpp
new file mode 100644
index 0000000..12a7978
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.06.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Wrong delimiter passed so expected nothing in the output
+ */
+
+USE test;
+
+CREATE EXTERNAL DATASET DatasetCopyWrong(ColumnType) USING %adapter%
+(
+  %template%,
+  ("header"="false"),
+  ("delimiter"=","),
+  %additional_Properties%,
+  ("definition"="%path_prefix%copy-to-result/csv/delimiter"),
+  ("format" = "csv"),
+  ("requireVersionChangeDetection"="false"),
+  ("include"="*.csv")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.07.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.07.query.sqlpp
new file mode 100644
index 0000000..7e6f1c5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/delimiter/delimiter.07.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT id, name, amount, accountNumber
+FROM DatasetCopyWrong d order by d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.01.ddl.sqlpp
new file mode 100644
index 0000000..80a8c34
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.01.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+CREATE TYPE ColumnType AS {
+  id: bigint,
+  name: string,
+  amount: float,
+  accountNumber: double
+};
+
+CREATE COLLECTION TestCollection(ColumnType) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.02.update.sqlpp
new file mode 100644
index 0000000..d57a311
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.02.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+USE test;
+
+INSERT INTO TestCollection({"id":1, "name":"Macbook1", "amount":123.2, "accountNumber":345.34});
+INSERT INTO TestCollection({"id":2, "name":"Macbook2", "amount":456.7, "accountNumber":123.45});
+INSERT INTO TestCollection({"id":3, "name":"Macbook3", "amount":789.1, "accountNumber":678.90});
+INSERT INTO TestCollection({"id":4, "name":"Macbook4", "amount":234.5, "accountNumber":567.89});
+INSERT INTO TestCollection({"id":5, "name":"Macbook5", "amount":876.5, "accountNumber":345.67});
+INSERT INTO TestCollection({"id":6, "name":"Macbook6", "amount":345.6, "accountNumber":987.65});
+INSERT INTO TestCollection({"id":7, "name":"Macbook7", "amount":678.9, "accountNumber":234.56});
+INSERT INTO TestCollection({"id":8, "name":"Macbook8", "amount":987.2, "accountNumber":789.12});
+INSERT INTO TestCollection({"id":9, "name":"Macbook9", "amount":543.2, "accountNumber":321.45});
+INSERT INTO TestCollection({"id":10, "name":"Macbook10", "amount":123.9, "accountNumber":654.32});
+INSERT INTO TestCollection({"id":11, "name":"Macbook11", "amount":567.8, "accountNumber":456.78});
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.03.update.sqlpp
new file mode 100644
index 0000000..df3ee42
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.03.update.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+USE test;
+
+COPY (
+   SELECT id, name, amount, accountNumber FROM TestCollection
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv", "header")
+AS (id bigint, name STRING, amount float, accountNumber double)
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"csv",
+    "delimiter":"|",
+    "header":"true"
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.04.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.04.ddl.sqlpp
new file mode 100644
index 0000000..2516a7f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.04.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+CREATE EXTERNAL DATASET DatasetCopy(ColumnType) USING %adapter%
+(
+  %template%,
+  ("header"="true"),
+  ("delimiter"="|"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%copy-to-result/csv/header"),
+  ("format" = "csv"),
+  ("requireVersionChangeDetection"="false"),
+  ("include"="*.csv")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.05.query.sqlpp
new file mode 100644
index 0000000..b407ec4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.05.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT id, name, amount, accountNumber
+FROM DatasetCopy d order by d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.10.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.10.update.sqlpp
new file mode 100644
index 0000000..157a9ff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.10.update.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Default Null Test
+ */
+
+USE test;
+
+COPY (
+   SELECT id, null name, amount, accountNumber FROM TestCollection
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv", "default", "null")
+AS (id bigint, name STRING, amount float, accountNumber double)
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"csv",
+    "delimiter":"|",
+    "header":"true"
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.11.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.11.ddl.sqlpp
new file mode 100644
index 0000000..0754048
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.11.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+CREATE EXTERNAL DATASET DatasetCopyDefaultNull(ColumnType) USING %adapter%
+(
+  %template%,
+  ("header"="true"),
+  ("delimiter"="|"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%copy-to-result/csv/default/null"),
+  ("format" = "csv"),
+  ("requireVersionChangeDetection"="false"),
+  ("include"="*.csv")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.12.query.sqlpp
new file mode 100644
index 0000000..bff57e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.12.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT id, name, amount, accountNumber
+FROM DatasetCopyDefaultNull dn order by dn.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.20.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.20.update.sqlpp
new file mode 100644
index 0000000..f02b7b8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.20.update.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+   SELECT id, null name, amount, accountNumber FROM TestCollection
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv", "custom", "null")
+AS (id bigint, name STRING, amount float, accountNumber double)
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"csv",
+    "delimiter":"|",
+    "header":"true",
+    "null":"IamNull"
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.21.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.21.ddl.sqlpp
new file mode 100644
index 0000000..29b53e6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.21.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Custom Null Test
+ */
+
+USE test;
+
+CREATE EXTERNAL DATASET DatasetCopyCustomNull(ColumnType) USING %adapter%
+(
+  %template%,
+  ("header"="true"),
+  ("delimiter"="|"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%copy-to-result/csv/custom/null"),
+  ("format" = "csv"),
+  ("requireVersionChangeDetection"="false"),
+  ("include"="*.csv")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.22.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.22.query.sqlpp
new file mode 100644
index 0000000..72f9ff3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.22.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT id, name, amount, accountNumber
+FROM DatasetCopyCustomNull dn order by dn.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.30.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.30.update.sqlpp
new file mode 100644
index 0000000..7d7d447
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.30.update.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+   SELECT id, null name, amount, accountNumber FROM TestCollection
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv", "notUnknown")
+AS (id bigint, name STRING not unknown, amount float, accountNumber double)
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"csv",
+    "delimiter":"|",
+    "header":"true",
+    "null":"IamNull"
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.31.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.31.ddl.sqlpp
new file mode 100644
index 0000000..c0f62b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.31.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Custom Null Test
+ */
+
+USE test;
+
+CREATE EXTERNAL DATASET DatasetCopyNotUnknown(ColumnType) USING %adapter%
+(
+  %template%,
+  ("header"="true"),
+  ("delimiter"="|"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%copy-to-result/csv/notUnknown"),
+  ("format" = "csv"),
+  ("requireVersionChangeDetection"="false"),
+  ("include"="*.csv")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.32.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.32.query.sqlpp
new file mode 100644
index 0000000..a6f1945
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/header/header.32.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT id, name, amount, accountNumber
+FROM DatasetCopyNotUnknown dn order by dn.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/null/null.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/null/null.01.ddl.sqlpp
new file mode 100644
index 0000000..80a8c34
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/null/null.01.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+CREATE TYPE ColumnType AS {
+  id: bigint,
+  name: string,
+  amount: float,
+  accountNumber: double
+};
+
+CREATE COLLECTION TestCollection(ColumnType) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/null/null.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/null/null.02.update.sqlpp
new file mode 100644
index 0000000..d57a311
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/null/null.02.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+USE test;
+
+INSERT INTO TestCollection({"id":1, "name":"Macbook1", "amount":123.2, "accountNumber":345.34});
+INSERT INTO TestCollection({"id":2, "name":"Macbook2", "amount":456.7, "accountNumber":123.45});
+INSERT INTO TestCollection({"id":3, "name":"Macbook3", "amount":789.1, "accountNumber":678.90});
+INSERT INTO TestCollection({"id":4, "name":"Macbook4", "amount":234.5, "accountNumber":567.89});
+INSERT INTO TestCollection({"id":5, "name":"Macbook5", "amount":876.5, "accountNumber":345.67});
+INSERT INTO TestCollection({"id":6, "name":"Macbook6", "amount":345.6, "accountNumber":987.65});
+INSERT INTO TestCollection({"id":7, "name":"Macbook7", "amount":678.9, "accountNumber":234.56});
+INSERT INTO TestCollection({"id":8, "name":"Macbook8", "amount":987.2, "accountNumber":789.12});
+INSERT INTO TestCollection({"id":9, "name":"Macbook9", "amount":543.2, "accountNumber":321.45});
+INSERT INTO TestCollection({"id":10, "name":"Macbook10", "amount":123.9, "accountNumber":654.32});
+INSERT INTO TestCollection({"id":11, "name":"Macbook11", "amount":567.8, "accountNumber":456.78});
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/null/null.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/null/null.03.update.sqlpp
new file mode 100644
index 0000000..b101ae5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/null/null.03.update.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+USE test;
+
+COPY (
+   SELECT id, null name, amount, accountNumber FROM TestCollection
+) toWriter
+TO S3
+PATH ("copy-to-result", "csv", "null")
+AS (id bigint, name STRING, amount float, accountNumber double)
+WITH {
+    "accessKeyId":"dummyAccessKey",
+    "secretAccessKey":"dummySecretKey",
+    "region":"us-west-2",
+    "serviceEndpoint":"http://127.0.0.1:8001",
+    "container":"playground",
+    "format":"csv",
+    "delimiter":"|",
+    "header":"true",
+    "null":"IamNull"
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/null/null.04.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/null/null.04.ddl.sqlpp
new file mode 100644
index 0000000..f3fdc90
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/null/null.04.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+CREATE EXTERNAL DATASET DatasetCopyNull(ColumnType) USING S3
+(
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("sessionToken"="dummySessionToken"),
+("header"="true"),
+("delimiter"="|"),
+("region"="us-west-2"),
+  ("serviceEndpoint"="http://127.0.0.1:8001"),
+  ("container"="playground"),
+  ("definition"="copy-to-result/csv/null"),
+  ("format" = "csv"),
+  ("requireVersionChangeDetection"="false"),
+  ("include"="*.csv")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/null/null.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/null/null.05.query.sqlpp
new file mode 100644
index 0000000..28beb5d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/null/null.05.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT id, name, amount, accountNumber
+FROM DatasetCopyNull d order by d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.01.ddl.sqlpp
new file mode 100644
index 0000000..9ee53a6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.01.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/* Currently quote and escape are not supported while creating external dataset
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+CREATE TYPE ColumnType AS {
+  id: bigint,
+  name: string,
+  amount: float,
+  accountNumber: double
+};
+
+CREATE COLLECTION TestCollection(ColumnType) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.02.update.sqlpp
new file mode 100644
index 0000000..dcfeef0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.02.update.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+USE test;
+
+INSERT INTO TestCollection({"id":1, "name":"Macbook1", "amount":123.2, "accountNumber":345.34});
+INSERT INTO TestCollection({"id":2, "name":"Macbook2", "amount":456.7, "accountNumber":123.45});
+INSERT INTO TestCollection({"id":3, "name":"Macbook3", "amount":789.1, "accountNumber":678.90});
+INSERT INTO TestCollection({"id":4, "name":"Mac|,book4", "amount":234.5, "accountNumber":567.89});
+
+COPY (
+   SELECT id, name, amount, accountNumber FROM TestCollection
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv", "escape")
+AS (id bigint, name STRING, amount float, accountNumber double)
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"csv",
+    "header":"true",
+    "escape":"|"
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.03.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.03.ddl.sqlpp
new file mode 100644
index 0000000..2ddd4c5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.03.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+CREATE EXTERNAL DATASET DatasetCopy(ColumnType) USING %adapter%
+(
+  %template%,
+  ("header"="true"),
+  ("escape"="|"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%copy-to-result/csv/escape"),
+  ("format" = "csv"),
+  ("requireVersionChangeDetection"="false"),
+  ("include"="*.csv")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.04.query.sqlpp
new file mode 100644
index 0000000..b407ec4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.04.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT id, name, amount, accountNumber
+FROM DatasetCopy d order by d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.11.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.11.update.sqlpp
new file mode 100644
index 0000000..94ce946
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.11.update.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+USE test;
+
+COPY (
+   SELECT id, name, amount, accountNumber FROM TestCollection
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv", "escape", "1")
+AS (id bigint, name STRING, amount float, accountNumber double)
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"csv",
+    "header":"true",
+    "escape":"|",
+    "quote": "NONE"
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.12.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.12.ddl.sqlpp
new file mode 100644
index 0000000..aaf9211
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.12.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+CREATE EXTERNAL DATASET DatasetCopy1(ColumnType) USING %adapter%
+(
+  %template%,
+  ("header"="true"),
+  ("escape"="|"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%copy-to-result/csv/escape/1"),
+  ("format" = "csv"),
+  ("requireVersionChangeDetection"="false"),
+  ("include"="*.csv")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.13.query.sqlpp
new file mode 100644
index 0000000..cc786e9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/quote-escape/quote-escape.13.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT id, name, amount, accountNumber
+FROM DatasetCopy1 d order by d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.01.ddl.sqlpp
new file mode 100644
index 0000000..1e43374
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.01.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+CREATE TYPE ColumnType AS {
+  id: bigint,
+  name: string?,
+  amount: float,
+  accountNumber: double
+};
+
+CREATE COLLECTION TestCollection(ColumnType) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.02.update.sqlpp
new file mode 100644
index 0000000..80b535c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.02.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+USE test;
+
+INSERT INTO TestCollection({"id":1, "name":"", "amount":123.2, "accountNumber":345.34});
+INSERT INTO TestCollection({"id":2, "name":"Macbook2", "amount":456.7, "accountNumber":123.45});
+INSERT INTO TestCollection({"id":3, "name":"Macbook3", "amount":789.1, "accountNumber":678.90});
+INSERT INTO TestCollection({"id":4, "name":"Macbook4", "amount":234.5, "accountNumber":567.89});
+INSERT INTO TestCollection({"id":5, "name":"Macbook5", "amount":876.5, "accountNumber":345.67});
+INSERT INTO TestCollection({"id":6, "name":"Macbook6", "amount":345.6, "accountNumber":987.65});
+INSERT INTO TestCollection({"id":7, "name":"Macbook7", "amount":678.9, "accountNumber":234.56});
+INSERT INTO TestCollection({"id":8, "name":"Macbook8", "amount":987.2, "accountNumber":789.12});
+INSERT INTO TestCollection({"id":9, "name":"Macbook9", "amount":543.2, "accountNumber":321.45});
+INSERT INTO TestCollection({"id":10, "name":"Macbook10", "amount":123.9, "accountNumber":654.32});
+INSERT INTO TestCollection({"id":11, "name":"Macbook11", "amount":567.8, "accountNumber":456.78});
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.03.update.sqlpp
new file mode 100644
index 0000000..b9a6e9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.03.update.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * This test handles the case with all the CSV knobs. Schema type is JSON style.
+ */
+
+USE test;
+
+COPY (
+   SELECT id, null name, amount, accountNumber FROM TestCollection
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv", "simple-csv", "1")
+TYPE ( {id: bigint, name: string?, amount: float, accountNumber: double} )
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"csv",
+    "delimiter":"|",
+    "header":"true",
+    "null":"IamNull",
+    "quote":"'",
+    "force-quote":"false",
+    "escape":"\\",
+    "empty_field_as_null":"true"
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.04.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.04.ddl.sqlpp
new file mode 100644
index 0000000..2c583ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.04.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+CREATE EXTERNAL DATASET DatasetCopy1(ColumnType) USING %adapter%
+(
+  %template%,
+  ("header"="true"),
+  ("delimiter"="|"),
+  ("quote"="'"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%copy-to-result/csv/simple-csv/1"),
+  ("format" = "csv"),
+  ("requireVersionChangeDetection"="false"),
+  ("include"="*.csv")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.05.query.sqlpp
new file mode 100644
index 0000000..cc786e9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.05.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT id, name, amount, accountNumber
+FROM DatasetCopy1 d order by d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.11.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.11.update.sqlpp
new file mode 100644
index 0000000..7e477be
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.11.update.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * This test handles the case with all the CSV knobs. Schema type is JSON style.
+ * Check for the case when putting missing as null values.
+ */
+
+USE test;
+
+COPY (
+   SELECT id, name, amount, accountNumber FROM TestCollection
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv", "simple-csv", "2")
+TYPE ( {id: bigint, name: string?, amount: float, accountNumber: double} )
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"csv",
+    "delimiter":"|",
+    "header":"true",
+    "null":"IamNull",
+    "quote":"'",
+    "force-quote":"false",
+    "escape":"\\",
+    "empty_field_as_null":"true"
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.12.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.12.ddl.sqlpp
new file mode 100644
index 0000000..5eca10c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.12.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+CREATE EXTERNAL DATASET DatasetCopy2(ColumnType) USING %adapter%
+(
+  %template%,
+  ("header"="true"),
+  ("delimiter"="|"),
+  ("quote"="'"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%copy-to-result/csv/simple-csv/2"),
+  ("format" = "csv"),
+  ("requireVersionChangeDetection"="false"),
+  ("include"="*.csv")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.13.query.sqlpp
new file mode 100644
index 0000000..547b512
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/simple-csv/simple-csv.13.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT id, name, amount, accountNumber
+FROM DatasetCopy2 d order by d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/type-mismatch/type-mismatch.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/type-mismatch/type-mismatch.01.ddl.sqlpp
new file mode 100644
index 0000000..6228097
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/type-mismatch/type-mismatch.01.ddl.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/type-mismatch/type-mismatch.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/type-mismatch/type-mismatch.02.update.sqlpp
new file mode 100644
index 0000000..d138157
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/type-mismatch/type-mismatch.02.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+USE test;
+
+COPY (
+   SELECT "123" as id
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv", "type-mismatch")
+AS (id bigint)
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"csv"
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/type-mismatch/type-mismatch.03.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/type-mismatch/type-mismatch.03.ddl.sqlpp
new file mode 100644
index 0000000..69d133d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/type-mismatch/type-mismatch.03.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+CREATE EXTERNAL DATASET DatasetCopy(id String) USING %adapter%
+(
+  %template%,
+  ("header"="false"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%copy-to-result/csv/type-mismatch"),
+  ("format" = "csv"),
+  ("requireVersionChangeDetection"="false"),
+  ("include"="*.csv")
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/type-mismatch/type-mismatch.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/type-mismatch/type-mismatch.04.query.sqlpp
new file mode 100644
index 0000000..8e2b639
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/csv/type-mismatch/type-mismatch.04.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT id
+FROM DatasetCopy c order by c.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.01.ddl.sqlpp
index 7207aa7..31db6b0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.01.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.01.ddl.sqlpp
@@ -20,13 +20,10 @@
 CREATE TYPE OpenType AS {
 };
 
-CREATE EXTERNAL DATASET Customer(OpenType) USING S3 (
-    ("accessKeyId"="dummyAccessKey"),
-    ("secretAccessKey"="dummySecretKey"),
-    ("region"="us-west-2"),
-    ("serviceEndpoint"="http://127.0.0.1:8001"),
-    ("container"="playground"),
-    ("definition"="external-filter/car/{company:string}/customer/{customer_id:int}"),
+CREATE EXTERNAL DATASET Customer(OpenType) USING %adapter% (
+    %template%,
+    %additional_Properties%,
+    ("definition"="%path_prefix%external-filter/car/{company:string}/customer/{customer_id:int}"),
     ("embed-filter-values" = "false"),
     ("format"="json")
 );
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.02.update.sqlpp
index 55916f3..5ee47cb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.02.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.02.update.sqlpp
@@ -18,14 +18,11 @@
  */
 
 COPY Customer c
-TO S3
-PATH ("copy-to-result", "default-namespace1")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "default-namespace1")
 WITH {
-    "accessKeyId":"dummyAccessKey",
-    "secretAccessKey":"dummySecretKey",
-    "region":"us-west-2",
-    "serviceEndpoint":"http://127.0.0.1:8001",
-    "container":"playground",
+    %template_colons%,
+    %additionalProperties%
     "format":"json"
 }
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.03.update.sqlpp
index edb038f..7fae07a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.03.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.03.update.sqlpp
@@ -18,18 +18,15 @@
  */
 
 COPY Customer AS c
-TO S3
-PATH ("copy-to-result/default-namespace2", company, "customer", customer_id)
+TO %adapter%
+PATH (%pathprefix% "copy-to-result/default-namespace2", company, "customer", customer_id)
 OVER (
    PARTITION BY c.company company,
                 c.customer_id customer_id
 )
 WITH {
-    "accessKeyId":"dummyAccessKey",
-    "secretAccessKey":"dummySecretKey",
-    "region":"us-west-2",
-    "serviceEndpoint":"http://127.0.0.1:8001",
-    "container":"playground",
+    %template_colons%,
+    %additionalProperties%
     "format":"json"
 }
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.04.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.04.update.sqlpp
index ee75bd7..8464574 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.04.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.04.update.sqlpp
@@ -21,18 +21,15 @@
    SELECT DISTINCT UPPERCASE(c.company) company, c.year
    FROM Customer c
 ) AS toWriter
-TO S3
-PATH ("copy-to-result/default-namespace3", company, year)
+TO %adapter%
+PATH (%pathprefix% "copy-to-result/default-namespace3", company, year)
 OVER (
    PARTITION BY toWriter.company company,
                 toWriter.year year
 )
 WITH {
-    "accessKeyId":"dummyAccessKey",
-    "secretAccessKey":"dummySecretKey",
-    "region":"us-west-2",
-    "serviceEndpoint":"http://127.0.0.1:8001",
-    "container":"playground",
+    %template_colons%,
+    %additionalProperties%
     "format":"json"
 }
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.05.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.05.ddl.sqlpp
index 4c7ba29..5e7de72 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.05.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/default-namespace/default-namespace.05.ddl.sqlpp
@@ -17,35 +17,26 @@
  * under the License.
  */
 
-CREATE EXTERNAL DATASET CustomerCopy1(OpenType) USING S3 (
-    ("accessKeyId"="dummyAccessKey"),
-    ("secretAccessKey"="dummySecretKey"),
-    ("region"="us-west-2"),
-    ("serviceEndpoint"="http://127.0.0.1:8001"),
-    ("container"="playground"),
-    ("definition"="copy-to-result/default-namespace1"),
+CREATE EXTERNAL DATASET CustomerCopy1(OpenType) USING %adapter% (
+    %template%,
+    %additional_Properties%,
+    ("definition"="%path_prefix%copy-to-result/default-namespace1"),
     ("embed-filter-values" = "false"),
     ("format"="json")
 );
 
-CREATE EXTERNAL DATASET CustomerCopy2(OpenType) USING S3 (
-    ("accessKeyId"="dummyAccessKey"),
-    ("secretAccessKey"="dummySecretKey"),
-    ("region"="us-west-2"),
-    ("serviceEndpoint"="http://127.0.0.1:8001"),
-    ("container"="playground"),
-    ("definition"="copy-to-result/default-namespace2/{company:string}/customer/{customer_id:int}"),
+CREATE EXTERNAL DATASET CustomerCopy2(OpenType) USING %adapter% (
+    %template%,
+    %additional_Properties%,
+    ("definition"="%path_prefix%copy-to-result/default-namespace2/{company:string}/customer/{customer_id:int}"),
     ("embed-filter-values" = "false"),
     ("format"="json")
 );
 
-CREATE EXTERNAL DATASET CustomerCopy3(OpenType) USING S3 (
-    ("accessKeyId"="dummyAccessKey"),
-    ("secretAccessKey"="dummySecretKey"),
-    ("region"="us-west-2"),
-    ("serviceEndpoint"="http://127.0.0.1:8001"),
-    ("container"="playground"),
-    ("definition"="copy-to-result/default-namespace3/{company:string}/{year:int}"),
+CREATE EXTERNAL DATASET CustomerCopy3(OpenType) USING %adapter% (
+    %template%,
+    %additional_Properties%,
+    ("definition"="%path_prefix%copy-to-result/default-namespace3/{company:string}/{year:int}"),
     ("embed-filter-values" = "false"),
     ("format"="json")
 );
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/bad-max-objects-per-file/empty-over.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/bad-max-objects-per-file/empty-over.01.ddl.sqlpp
index b68c38b..a384802 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/bad-max-objects-per-file/empty-over.01.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/bad-max-objects-per-file/empty-over.01.ddl.sqlpp
@@ -24,13 +24,10 @@
 CREATE TYPE OpenType AS {
 };
 
-CREATE EXTERNAL DATASET Customer(OpenType) USING S3 (
-    ("accessKeyId"="dummyAccessKey"),
-    ("secretAccessKey"="dummySecretKey"),
-    ("region"="us-west-2"),
-    ("serviceEndpoint"="http://127.0.0.1:8001"),
-    ("container"="playground"),
-    ("definition"="external-filter/car/{company:string}/customer/{customer_id:int}"),
+CREATE EXTERNAL DATASET Customer(OpenType) USING %adapter% (
+    %template%,
+    %additional_Properties%,
+    ("definition"="%path_prefix%external-filter/car/{company:string}/customer/{customer_id:int}"),
     ("embed-filter-values" = "false"),
     ("format"="json")
 );
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/bad-max-objects-per-file/empty-over.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/bad-max-objects-per-file/empty-over.02.update.sqlpp
index 02f6c96..b7dce3a4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/bad-max-objects-per-file/empty-over.02.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/bad-max-objects-per-file/empty-over.02.update.sqlpp
@@ -20,14 +20,11 @@
 USE test;
 
 COPY Customer c
-TO S3
-PATH ("copy-to-result-with-over")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result-with-over")
 WITH {
-    "accessKeyId":"dummyAccessKey",
-    "secretAccessKey":"dummySecretKey",
-    "region":"us-west-2",
-    "serviceEndpoint":"http://127.0.0.1:8001",
-    "container":"playground",
+    %template_colons%,
+    %additionalProperties%
     "format":"json",
     "max-objects-per-file": 2
 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/bad-max-objects-per-file/empty-over.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/bad-max-objects-per-file/empty-over.03.update.sqlpp
index c0add71..33e3457 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/bad-max-objects-per-file/empty-over.03.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/bad-max-objects-per-file/empty-over.03.update.sqlpp
@@ -20,14 +20,11 @@
 USE test;
 
 COPY Customer c
-TO S3
-PATH ("copy-to-result-with-over")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result-with-over")
 WITH {
-    "accessKeyId":"dummyAccessKey",
-    "secretAccessKey":"dummySecretKey",
-    "region":"us-west-2",
-    "serviceEndpoint":"http://127.0.0.1:8001",
-    "container":"playground",
+    %template_colons%,
+    %additionalProperties%
     "format":"json",
     "max-objects-per-file": "hello"
 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.01.ddl.sqlpp
new file mode 100644
index 0000000..591c949
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.01.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test if exists;
+CREATE DATAVERSE test;
+USE test;
+
+
+CREATE TYPE ColumnType1 AS {
+  id: integer
+};
+
+CREATE COLLECTION TestCollection(ColumnType1) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.02.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.02.ddl.sqlpp
new file mode 100644
index 0000000..db7509f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.02.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+   select c.* from TestCollection c
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv-error-checks2")
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"csv"
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.03.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.03.ddl.sqlpp
new file mode 100644
index 0000000..cc02e20
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.03.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+   select c.* from TestCollection c
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv-error-checks3")
+AS (id wrongDataType)
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"csv"
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.04.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.04.ddl.sqlpp
new file mode 100644
index 0000000..78f752d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.04.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+   select c.* from TestCollection c
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv-error-checks3")
+AS (id bigint)
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"csv",
+    "quote": "ABCD"
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.05.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.05.ddl.sqlpp
new file mode 100644
index 0000000..41e1c83
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.05.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+   select c.* from TestCollection c
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv-error-checks3")
+AS (id bigint)
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"csv",
+    "delimiter": "wrongDelimiter"
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.06.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.06.ddl.sqlpp
new file mode 100644
index 0000000..4d20c23
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.06.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+   select c.* from TestCollection c
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv-error-checks3")
+AS (id bigint)
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"csv",
+    "escape": "wrongEscape"
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.07.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.07.ddl.sqlpp
new file mode 100644
index 0000000..a53e2ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.07.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+   select c.* from TestCollection c
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv-error-checks3")
+AS (id bigint)
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"csv",
+    "record-delimiter": "ABCD"
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.08.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.08.ddl.sqlpp
new file mode 100644
index 0000000..5d68650
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.08.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+   select c.* from TestCollection c
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv-error-checks3")
+TYPE ( { id : int,  name : { first :  [ string ] } } )
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"csv",
+    "record-delimiter": ","
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.09.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.09.ddl.sqlpp
new file mode 100644
index 0000000..ceab3aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.09.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+   select c.* from TestCollection c
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv-error-checks3")
+TYPE ( { id : int,  name : [ string ] } )
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"csv",
+    "record-delimiter": ","
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.10.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.10.ddl.sqlpp
new file mode 100644
index 0000000..6a38cea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/csv-error-checks/csv-error-checks.10.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+   select c.* from TestCollection c
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "csv-error-checks3")
+TYPE ( { id : int,  name : string } )
+AS ( id string )
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"csv",
+    "record-delimiter": ","
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/early-missing/early-missing.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/early-missing/early-missing.01.ddl.sqlpp
index b68c38b..a384802 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/early-missing/early-missing.01.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/early-missing/early-missing.01.ddl.sqlpp
@@ -24,13 +24,10 @@
 CREATE TYPE OpenType AS {
 };
 
-CREATE EXTERNAL DATASET Customer(OpenType) USING S3 (
-    ("accessKeyId"="dummyAccessKey"),
-    ("secretAccessKey"="dummySecretKey"),
-    ("region"="us-west-2"),
-    ("serviceEndpoint"="http://127.0.0.1:8001"),
-    ("container"="playground"),
-    ("definition"="external-filter/car/{company:string}/customer/{customer_id:int}"),
+CREATE EXTERNAL DATASET Customer(OpenType) USING %adapter% (
+    %template%,
+    %additional_Properties%,
+    ("definition"="%path_prefix%external-filter/car/{company:string}/customer/{customer_id:int}"),
     ("embed-filter-values" = "false"),
     ("format"="json")
 );
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/early-missing/early-missing.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/early-missing/early-missing.02.update.sqlpp
index 434801b..5c035cd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/early-missing/early-missing.02.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/early-missing/early-missing.02.update.sqlpp
@@ -20,14 +20,11 @@
 USE test;
 
 COPY Customer c
-TO S3
+TO %adapter%
 PATH (MISSING)
 WITH {
-    "accessKeyId":"dummyAccessKey",
-    "secretAccessKey":"dummySecretKey",
-    "region":"us-west-2",
-    "serviceEndpoint":"http://127.0.0.1:8001",
-    "container":"playground",
+    %template_colons%,
+    %additionalProperties%
     "format":"json"
 }
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/early-missing/early-missing.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/early-missing/early-missing.03.update.sqlpp
index d02486b..af3f9f2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/early-missing/early-missing.03.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/early-missing/early-missing.03.update.sqlpp
@@ -20,14 +20,11 @@
 USE test;
 
 COPY Customer c
-TO S3
+TO %adapter%
 PATH (1 + "hello world")
 WITH {
-    "accessKeyId":"dummyAccessKey",
-    "secretAccessKey":"dummySecretKey",
-    "region":"us-west-2",
-    "serviceEndpoint":"http://127.0.0.1:8001",
-    "container":"playground",
+    %template_colons%,
+    %additionalProperties%
     "format":"json"
 }
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/empty-over/empty-over.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/empty-over/empty-over.01.ddl.sqlpp
index b68c38b..a384802 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/empty-over/empty-over.01.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/empty-over/empty-over.01.ddl.sqlpp
@@ -24,13 +24,10 @@
 CREATE TYPE OpenType AS {
 };
 
-CREATE EXTERNAL DATASET Customer(OpenType) USING S3 (
-    ("accessKeyId"="dummyAccessKey"),
-    ("secretAccessKey"="dummySecretKey"),
-    ("region"="us-west-2"),
-    ("serviceEndpoint"="http://127.0.0.1:8001"),
-    ("container"="playground"),
-    ("definition"="external-filter/car/{company:string}/customer/{customer_id:int}"),
+CREATE EXTERNAL DATASET Customer(OpenType) USING %adapter% (
+    %template%,
+    %additional_Properties%,
+    ("definition"="%path_prefix%external-filter/car/{company:string}/customer/{customer_id:int}"),
     ("embed-filter-values" = "false"),
     ("format"="json")
 );
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/empty-over/empty-over.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/empty-over/empty-over.02.update.sqlpp
index cc0c4b4..2df776c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/empty-over/empty-over.02.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/empty-over/empty-over.02.update.sqlpp
@@ -20,15 +20,12 @@
 USE test;
 
 COPY Customer c
-TO S3
-PATH ("copy-to-result-with-over")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result-with-over")
 OVER()
 WITH {
-    "accessKeyId":"dummyAccessKey",
-    "secretAccessKey":"dummySecretKey",
-    "region":"us-west-2",
-    "serviceEndpoint":"http://127.0.0.1:8001",
-    "container":"playground",
+    %template_colons%,
+    %additionalProperties%
     "format":"json"
 }
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/non-empty-folder/non-empty-folder.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/non-empty-folder/non-empty-folder.01.ddl.sqlpp
index b68c38b..a384802 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/non-empty-folder/non-empty-folder.01.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/non-empty-folder/non-empty-folder.01.ddl.sqlpp
@@ -24,13 +24,10 @@
 CREATE TYPE OpenType AS {
 };
 
-CREATE EXTERNAL DATASET Customer(OpenType) USING S3 (
-    ("accessKeyId"="dummyAccessKey"),
-    ("secretAccessKey"="dummySecretKey"),
-    ("region"="us-west-2"),
-    ("serviceEndpoint"="http://127.0.0.1:8001"),
-    ("container"="playground"),
-    ("definition"="external-filter/car/{company:string}/customer/{customer_id:int}"),
+CREATE EXTERNAL DATASET Customer(OpenType) USING %adapter% (
+    %template%,
+    %additional_Properties%,
+    ("definition"="%path_prefix%external-filter/car/{company:string}/customer/{customer_id:int}"),
     ("embed-filter-values" = "false"),
     ("format"="json")
 );
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/non-empty-folder/non-empty-folder.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/non-empty-folder/non-empty-folder.02.update.sqlpp
index a0e04db..be98bcd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/non-empty-folder/non-empty-folder.02.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/non-empty-folder/non-empty-folder.02.update.sqlpp
@@ -20,14 +20,11 @@
 USE test;
 
 COPY Customer c
-TO S3
-PATH ("copy-to-result", "duplicate-write")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "duplicate-write")
 WITH {
-    "accessKeyId":"dummyAccessKey",
-    "secretAccessKey":"dummySecretKey",
-    "region":"us-west-2",
-    "serviceEndpoint":"http://127.0.0.1:8001",
-    "container":"playground",
+    %template_colons%,
+    %additionalProperties%
     "format":"json"
 }
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/non-empty-folder/non-empty-folder.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/non-empty-folder/non-empty-folder.03.update.sqlpp
index a0e04db..be98bcd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/non-empty-folder/non-empty-folder.03.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/non-empty-folder/non-empty-folder.03.update.sqlpp
@@ -20,14 +20,11 @@
 USE test;
 
 COPY Customer c
-TO S3
-PATH ("copy-to-result", "duplicate-write")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "duplicate-write")
 WITH {
-    "accessKeyId":"dummyAccessKey",
-    "secretAccessKey":"dummySecretKey",
-    "region":"us-west-2",
-    "serviceEndpoint":"http://127.0.0.1:8001",
-    "container":"playground",
+    %template_colons%,
+    %additionalProperties%
     "format":"json"
 }
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.01.ddl.sqlpp
new file mode 100644
index 0000000..0bfc4eea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.01.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test if exists;
+CREATE DATAVERSE test;
+USE test;
+
+
+CREATE TYPE ColumnType1 AS {
+  id: int
+};
+
+CREATE COLLECTION TestCollection(ColumnType1) PRIMARY KEY id;
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.03.update.sqlpp
new file mode 100644
index 0000000..926dfaa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.03.update.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+insert into TestCollection({"id":2, "name": "John", "nested" : { "first" : "john" , "second":"JOHN" }  });
+
+COPY (
+   select c.* from TestCollection c
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks3")
+TYPE ( {id : string, name : string, nested: { first : string, second : string} }
+)
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"parquet"
+}
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.04.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.04.update.sqlpp
new file mode 100644
index 0000000..fae01f8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.04.update.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+
+COPY (
+   select c.* from TestCollection c
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks4")
+TYPE ( {id : bigint,name : string, nested: { first : string } } )
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"parquet"
+}
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.05.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.05.update.sqlpp
new file mode 100644
index 0000000..21c91dc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.05.update.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+COPY (
+   select c.* from TestCollection c
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks5")
+TYPE ( {  id : bigint, name : string, nested : string } )
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"parquet"
+}
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.06.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.06.update.sqlpp
new file mode 100644
index 0000000..2508c5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.06.update.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+COPY (
+   select c.* from TestCollection c
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks6")
+TYPE ( {id : bigint,name : {  first : string },nested:{first : string,second : string} } )
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"parquet"
+}
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.07.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.07.update.sqlpp
new file mode 100644
index 0000000..bb8a348
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.07.update.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+COPY (
+   select c.* from TestCollection c
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks7")
+TYPE ( {id:int} )
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"parquet",
+    "row-group-size":"random"
+}
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.08.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.08.update.sqlpp
new file mode 100644
index 0000000..ac7050c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.08.update.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+COPY (
+   select c.* from TestCollection c
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks8")
+TYPE ( {id:int} )
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"parquet",
+    "page-size":"random"
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.09.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.09.update.sqlpp
new file mode 100644
index 0000000..a2ae2e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.09.update.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+COPY (
+   select c.* from TestCollection c
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks9")
+TYPE ( { name:string } )
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"parquet",
+    "compression":"rar"
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.10.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.10.update.sqlpp
new file mode 100644
index 0000000..0069616
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.10.update.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+   select c.* from TestCollection c
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks10")
+TYPE ( { name: } )
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"parquet",
+    "schema":"message schema{"
+}
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.11.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.11.update.sqlpp
new file mode 100644
index 0000000..0f9fc7d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.11.update.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+   select c.* from TestCollection c
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks11")
+TYPE ( { id : int , name : binary } )
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"parquet"
+}
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.12.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.12.update.sqlpp
new file mode 100644
index 0000000..2531328
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.12.update.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+   select c.* from TestCollection c
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks12")
+TYPE ( { id :  int, name : string } )
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"parquet",
+    "version" : 3
+}
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.13.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.13.update.sqlpp
new file mode 100644
index 0000000..85bec05
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.13.update.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+   select c.* from TestCollection c
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks13")
+TYPE ( { id :  int,  name : string, list : [string]} )
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"parquet",
+    "max-schemas" : "yvghc"
+}
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.14.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.14.update.sqlpp
new file mode 100644
index 0000000..4614ead
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.14.update.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+   select c.* from TestCollection c
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks14")
+TYPE ( {id :  int, name : string, l : int } )
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"parquet",
+    "max-schemas" : "15"
+}
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.15.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.15.update.sqlpp
new file mode 100644
index 0000000..be198a2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.15.update.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+
+
+
+
+insert into TestCollection({"id":1, "name": []});
+
+
+
+COPY (
+select c.* from TestCollection c
+    ) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks15")
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"parquet"
+    }
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.16.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.16.update.sqlpp
new file mode 100644
index 0000000..ed2d0d5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/parquet-error-checks/parquet-error-checks.16.update.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+
+
+insert into TestCollection({"id":10, "name": 1 });
+insert into TestCollection({"id":12, "name": [1] });
+insert into TestCollection({"id":15, "name": [[1]] });
+insert into TestCollection({"id":16, "name": [[[1]]] });
+insert into TestCollection({"id":17, "name": [[[[1]]]] });
+insert into TestCollection({"id":18, "name": [[[[[1]]]]] });
+insert into TestCollection({"id":19, "name": [[[[[[1]]]]]] });
+
+
+
+COPY (
+select id,name from TestCollection c
+    ) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-error-checks16")
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"parquet",
+    "max-schemas" : "2"
+    }
+
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/supported-adapter-format-compression/supported-adapters.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/supported-adapter-format-compression/supported-adapters.01.ddl.sqlpp
index b68c38b..a384802 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/supported-adapter-format-compression/supported-adapters.01.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/supported-adapter-format-compression/supported-adapters.01.ddl.sqlpp
@@ -24,13 +24,10 @@
 CREATE TYPE OpenType AS {
 };
 
-CREATE EXTERNAL DATASET Customer(OpenType) USING S3 (
-    ("accessKeyId"="dummyAccessKey"),
-    ("secretAccessKey"="dummySecretKey"),
-    ("region"="us-west-2"),
-    ("serviceEndpoint"="http://127.0.0.1:8001"),
-    ("container"="playground"),
-    ("definition"="external-filter/car/{company:string}/customer/{customer_id:int}"),
+CREATE EXTERNAL DATASET Customer(OpenType) USING %adapter% (
+    %template%,
+    %additional_Properties%,
+    ("definition"="%path_prefix%external-filter/car/{company:string}/customer/{customer_id:int}"),
     ("embed-filter-values" = "false"),
     ("format"="json")
 );
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/supported-adapter-format-compression/supported-adapters.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/supported-adapter-format-compression/supported-adapters.03.update.sqlpp
index 2e227d6..2c75a42 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/supported-adapter-format-compression/supported-adapters.03.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/supported-adapter-format-compression/supported-adapters.03.update.sqlpp
@@ -20,15 +20,12 @@
 USE test;
 
 COPY Customer c
-TO S3
-PATH ("copy-to-result")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result")
 WITH {
-    "accessKeyId":"dummyAccessKey",
-    "secretAccessKey":"dummySecretKey",
-    "region":"us-west-2",
-    "serviceEndpoint":"http://127.0.0.1:8001",
-    "container":"playground",
-    "format":"csv"
+    %template_colons%,
+    %additionalProperties%
+    "format":"avro"
 }
 
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/supported-adapter-format-compression/supported-adapters.04.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/supported-adapter-format-compression/supported-adapters.04.update.sqlpp
index 38934b9..6969404 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/supported-adapter-format-compression/supported-adapters.04.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/negative/supported-adapter-format-compression/supported-adapters.04.update.sqlpp
@@ -20,14 +20,11 @@
 USE test;
 
 COPY Customer c
-TO S3
-PATH ("copy-to-result")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result")
 WITH {
-    "accessKeyId":"dummyAccessKey",
-    "secretAccessKey":"dummySecretKey",
-    "region":"us-west-2",
-    "serviceEndpoint":"http://127.0.0.1:8001",
-    "container":"playground",
+    %template_colons%,
+    %additionalProperties%
     "format":"json",
     "compression": "rar"
 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.01.ddl.sqlpp
index b68c38b..a384802 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.01.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.01.ddl.sqlpp
@@ -24,13 +24,10 @@
 CREATE TYPE OpenType AS {
 };
 
-CREATE EXTERNAL DATASET Customer(OpenType) USING S3 (
-    ("accessKeyId"="dummyAccessKey"),
-    ("secretAccessKey"="dummySecretKey"),
-    ("region"="us-west-2"),
-    ("serviceEndpoint"="http://127.0.0.1:8001"),
-    ("container"="playground"),
-    ("definition"="external-filter/car/{company:string}/customer/{customer_id:int}"),
+CREATE EXTERNAL DATASET Customer(OpenType) USING %adapter% (
+    %template%,
+    %additional_Properties%,
+    ("definition"="%path_prefix%external-filter/car/{company:string}/customer/{customer_id:int}"),
     ("embed-filter-values" = "false"),
     ("format"="json")
 );
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.02.update.sqlpp
index 66bb709..b528721 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.02.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.02.update.sqlpp
@@ -20,17 +20,14 @@
 USE test;
 
 COPY Customer c
-TO S3
-PATH ("copy-to-result", "order-by1")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "order-by1")
 -- We do not have a way to determine the output is sorted
 -- However, this test ensures that there's no regressions when having ORDER BY only in the OVER-clause
 OVER (ORDER BY c.company)
 WITH {
-    "accessKeyId":"dummyAccessKey",
-    "secretAccessKey":"dummySecretKey",
-    "region":"us-west-2",
-    "serviceEndpoint":"http://127.0.0.1:8001",
-    "container":"playground",
+    %template_colons%,
+    %additionalProperties%
     "format":"json"
 }
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.03.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.03.ddl.sqlpp
index 0380332..182fe2d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.03.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.03.ddl.sqlpp
@@ -19,13 +19,10 @@
 
 USE test;
 
-CREATE EXTERNAL DATASET CustomerCopy(OpenType) USING S3 (
-    ("accessKeyId"="dummyAccessKey"),
-    ("secretAccessKey"="dummySecretKey"),
-    ("region"="us-west-2"),
-    ("serviceEndpoint"="http://127.0.0.1:8001"),
-    ("container"="playground"),
-    ("definition"="copy-to-result/order-by1"),
+CREATE EXTERNAL DATASET CustomerCopy(OpenType) USING %adapter% (
+    %template%,
+    %additional_Properties%,
+    ("definition"="%path_prefix%copy-to-result/order-by1"),
     ("embed-filter-values" = "false"),
     ("format"="json")
 );
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.20.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.20.update.sqlpp
index 1fae58f..fa7ecec 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.20.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.20.update.sqlpp
@@ -23,17 +23,14 @@
    SELECT c.company make
    FROM Customer c
 ) x
-TO S3
-PATH ("copy-to-result", "order-by2")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "order-by2")
 -- We do not have a way to determine the output is sorted
 -- However, this test ensures that there's no regressions when having ORDER BY only in the OVER-clause
 OVER (ORDER BY x.make)
 WITH {
-    "accessKeyId":"dummyAccessKey",
-    "secretAccessKey":"dummySecretKey",
-    "region":"us-west-2",
-    "serviceEndpoint":"http://127.0.0.1:8001",
-    "container":"playground",
+    %template_colons%,
+    %additionalProperties%
     "format":"json"
 }
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.21.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.21.ddl.sqlpp
index fab6964..7c38e71 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.21.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.21.ddl.sqlpp
@@ -19,13 +19,10 @@
 
 USE test;
 
-CREATE EXTERNAL DATASET CustomerCarMake1(OpenType) USING S3 (
-    ("accessKeyId"="dummyAccessKey"),
-    ("secretAccessKey"="dummySecretKey"),
-    ("region"="us-west-2"),
-    ("serviceEndpoint"="http://127.0.0.1:8001"),
-    ("container"="playground"),
-    ("definition"="copy-to-result/order-by2"),
+CREATE EXTERNAL DATASET CustomerCarMake1(OpenType) USING %adapter% (
+    %template%,
+    %additional_Properties%,
+    ("definition"="%path_prefix%copy-to-result/order-by2"),
     ("embed-filter-values" = "false"),
     ("format"="json")
 );
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.30.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.30.update.sqlpp
index dc0a631..b3616f0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.30.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.30.update.sqlpp
@@ -23,17 +23,14 @@
    SELECT c.company make
    FROM Customer c
 ) x
-TO S3
-PATH ("copy-to-result", "order-by3")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "order-by3")
 -- We do not have a way to determine the output is sorted
 -- However, this test ensures that there's no regressions when having ORDER BY only in the OVER-clause
 OVER (ORDER BY x.make)
 WITH {
-    "accessKeyId":"dummyAccessKey",
-    "secretAccessKey":"dummySecretKey",
-    "region":"us-west-2",
-    "serviceEndpoint":"http://127.0.0.1:8001",
-    "container":"playground",
+    %template_colons%,
+    %additionalProperties%
     "format":"json"
 }
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.31.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.31.ddl.sqlpp
index e3369be..9e800c9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.31.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/order-by/order-by.31.ddl.sqlpp
@@ -19,13 +19,10 @@
 
 USE test;
 
-CREATE EXTERNAL DATASET CustomerCarMake2(OpenType) USING S3 (
-    ("accessKeyId"="dummyAccessKey"),
-    ("secretAccessKey"="dummySecretKey"),
-    ("region"="us-west-2"),
-    ("serviceEndpoint"="http://127.0.0.1:8001"),
-    ("container"="playground"),
-    ("definition"="copy-to-result/order-by2"),
+CREATE EXTERNAL DATASET CustomerCarMake2(OpenType) USING %adapter% (
+    %template%,
+    %additional_Properties%,
+    ("definition"="%path_prefix%copy-to-result/order-by2"),
     ("embed-filter-values" = "false"),
     ("format"="json")
 );
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-cover-data-types/parquet-cover-data-types.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-cover-data-types/parquet-cover-data-types.01.ddl.sqlpp
new file mode 100644
index 0000000..56e79c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-cover-data-types/parquet-cover-data-types.01.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test if exists;
+CREATE DATAVERSE test;
+USE test;
+
+
+CREATE TYPE ColumnType1 AS {
+  id: integer,
+  name : string
+};
+
+CREATE COLLECTION TestCollection(ColumnType1) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-cover-data-types/parquet-cover-data-types.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-cover-data-types/parquet-cover-data-types.02.update.sqlpp
new file mode 100644
index 0000000..ec1ac0c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-cover-data-types/parquet-cover-data-types.02.update.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : create a dataset using year-month-duration as the primary key
+ * Expected Res : Success
+ * Date         : 7 May 2013
+ * Issue        : 363
+ */
+
+use test;
+/*
+insert into TestCollection({"id":`year-month-duration`("P16Y"), "name": "John"});
+insert into TestCollection({"id":`day-time-duration`("-P3829H849.392S"), "name": "Alex"});
+*/
+
+insert into TestCollection({"id":18, "name": "Virat" , "dateType":date("1988-11-05"), "timeType": time("03:10:00.493Z") , "boolType" : false , "doubleType" : 0.75, "datetimeType" : datetime("1900-02-01T00:00:00") });
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-cover-data-types/parquet-cover-data-types.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-cover-data-types/parquet-cover-data-types.03.update.sqlpp
new file mode 100644
index 0000000..a85e188
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-cover-data-types/parquet-cover-data-types.03.update.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+   select c.* from TestCollection c
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-cover-data-types")
+TYPE ( {   name : string,  id : int,  dateType : date,   timeType : time,  boolType : boolean,   doubleType : double,  datetimeType : datetime   } )
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"parquet"
+};
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-cover-data-types/parquet-cover-data-types.04.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-cover-data-types/parquet-cover-data-types.04.ddl.sqlpp
new file mode 100644
index 0000000..54c9373
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-cover-data-types/parquet-cover-data-types.04.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+CREATE TYPE ColumnType2 AS {
+};
+
+
+
+CREATE EXTERNAL DATASET TestDataset(ColumnType2) USING %adapter%
+(
+  %template%,
+  %additional_Properties%,
+  ("definition"="%path_prefix%copy-to-result/parquet-cover-data-types/"),
+  ("include"="*.parquet"),
+  ("requireVersionChangeDetection"="false"),
+  ("format" = "parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-cover-data-types/parquet-cover-data-types.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-cover-data-types/parquet-cover-data-types.05.query.sqlpp
new file mode 100644
index 0000000..b03fc5e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-cover-data-types/parquet-cover-data-types.05.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT c.*
+FROM TestDataset c
+ORDER BY c.id;
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-empty-array/parquet-empty-array.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-empty-array/parquet-empty-array.01.ddl.sqlpp
new file mode 100644
index 0000000..6be9489
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-empty-array/parquet-empty-array.01.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test if exists;
+CREATE DATAVERSE test;
+USE test;
+
+
+CREATE TYPE ColumnType1 AS {
+    id: integer
+};
+
+CREATE COLLECTION TestCollection(ColumnType1) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-empty-array/parquet-empty-array.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-empty-array/parquet-empty-array.02.update.sqlpp
new file mode 100644
index 0000000..d569ee5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-empty-array/parquet-empty-array.02.update.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+INSERT INTO TestCollection {"id":2,"name":{"first":["power","star"]}};
+INSERT INTO TestCollection {"id":5,"name":{"first":[]}};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-empty-array/parquet-empty-array.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-empty-array/parquet-empty-array.03.update.sqlpp
new file mode 100644
index 0000000..0f33fcc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-empty-array/parquet-empty-array.03.update.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+select c.* from TestCollection c
+    ) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-empty-array")
+TYPE ( { id : int,  name : { first :  [ string ] } } )
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"parquet"
+    };
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-empty-array/parquet-empty-array.04.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-empty-array/parquet-empty-array.04.ddl.sqlpp
new file mode 100644
index 0000000..93aa8b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-empty-array/parquet-empty-array.04.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+CREATE TYPE ColumnType2 AS {
+    };
+
+
+
+CREATE EXTERNAL DATASET TestDataset(ColumnType2) USING %adapter%
+(
+  %template%,
+  %additional_Properties%,
+  ("definition"="%path_prefix%copy-to-result/parquet-empty-array/"),
+  ("include"="*.parquet"),
+  ("requireVersionChangeDetection"="false"),
+  ("format" = "parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-empty-array/parquet-empty-array.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-empty-array/parquet-empty-array.05.query.sqlpp
new file mode 100644
index 0000000..b03fc5e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-empty-array/parquet-empty-array.05.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT c.*
+FROM TestDataset c
+ORDER BY c.id;
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-heterogeneous/parquet-heterogeneous.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-heterogeneous/parquet-heterogeneous.01.ddl.sqlpp
new file mode 100644
index 0000000..00d5c49
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-heterogeneous/parquet-heterogeneous.01.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test if exists;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE ColumnType1 AS {
+ id: int
+};
+
+
+CREATE COLLECTION TestCollection(ColumnType1) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-heterogeneous/parquet-heterogeneous.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-heterogeneous/parquet-heterogeneous.02.update.sqlpp
new file mode 100644
index 0000000..308d3b9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-heterogeneous/parquet-heterogeneous.02.update.sqlpp
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+
+
+-- INSERT INTO TestCollection( {"id":2,"name":{"first":"virat"}});
+-- INSERT INTO TestCollection( {"id":5,"name":{"first":"virat"}});
+-- INSERT INTO TestCollection( {"id":8,"name":{"first":"virat"}});
+-- INSERT INTO TestCollection( {"id":10,"name":{"first":"virat"}});
+-- INSERT INTO TestCollection( {"id":12,"name":{"first":"virat"}});
+-- INSERT INTO TestCollection( {"id":15,"name":{"first":"virat"}});
+-- INSERT INTO TestCollection( {"id":17,"name":{"first":"virat"}});
+-- INSERT INTO TestCollection( {"id":20,"name":{"first":"virat"}});
+-- INSERT INTO TestCollection( {"id":21,"name":{"first":"virat"}});
+-- INSERT INTO TestCollection( {"id":27,"name":{"first":"virat"}});
+-- INSERT INTO TestCollection( {"id":28,"name":{"first":"virat"}});
+
+
+insert into TestCollection({"id":2});
+insert into TestCollection({"id":5,"name":"virat"});
+insert into TestCollection({"id":8,"name":{"first":"virat"}});
+insert into TestCollection({"id":10,"name":{"first":"virat"},"age":18});
+insert into TestCollection({"id":12,"name":123});
+insert into TestCollection({"id":15,"name":[123,456]});
+insert into TestCollection({"id":17,"name":765});
+insert into TestCollection({"id":20,"name":[789]});
+insert into TestCollection({"id":21,"name":[{"first":"virat"}]});
+insert into TestCollection({"id":27,"name":[{"first":"virat","second":"kohli"}]});
+insert into TestCollection({"id":28,"name":{"first":"virat"}});
+
+
+
+-- INSERT INTO TestCollection( {"id":1,"name":{"first":"virat"}});
+-- INSERT INTO TestCollection( {"id":2,"name":{"first":"virat"}});
+-- INSERT INTO TestCollection( {"id":3,"name":{"first":"virat"}});
+-- INSERT INTO TestCollection( {"id":4,"name":{"first":"virat"}});
+-- INSERT INTO TestCollection( {"id":5,"name":{"first":"virat"}});
+-- INSERT INTO TestCollection( {"id":6,"name":{"first":"virat"}});
+-- INSERT INTO TestCollection( {"id":7,"name":{"first":"virat"}});
+-- INSERT INTO TestCollection( {"id":8,"name":{"first":"virat"}});
+-- INSERT INTO TestCollection( {"id":9,"name":{"first":"virat"}});
+-- INSERT INTO TestCollection( {"id":10,"name":{"first":"virat"}});
+-- INSERT INTO TestCollection( {"id":11,"name":{"first":"virat"}});
+-- INSERT INTO TestCollection( {"id":12,"name":{"first":"virat"}});
+-- INSERT INTO TestCollection( {"id":13,"name":{"first":"virat"}});
+-- INSERT INTO TestCollection( {"id":14,"name":{"first":"virat"}});
+-- INSERT INTO TestCollection( {"id":15,"name":{"first":"virat"}});
+-- INSERT INTO TestCollection( {"id":16,"name":{"first":"virat"}});
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-heterogeneous/parquet-heterogeneous.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-heterogeneous/parquet-heterogeneous.03.update.sqlpp
new file mode 100644
index 0000000..ea9c327
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-heterogeneous/parquet-heterogeneous.03.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+   select c.* from TestCollection c
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-heterogeneous")
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"parquet"
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-heterogeneous/parquet-heterogeneous.04.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-heterogeneous/parquet-heterogeneous.04.ddl.sqlpp
new file mode 100644
index 0000000..0494045
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-heterogeneous/parquet-heterogeneous.04.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+CREATE TYPE ColumnType2 AS {
+};
+
+
+
+CREATE EXTERNAL DATASET TestDataset(ColumnType2) USING %adapter%
+(
+  %template%,
+  %additional_Properties%,
+  ("definition"="%path_prefix%copy-to-result/parquet-heterogeneous/"),
+  ("include"="*.parquet"),
+  ("requireVersionChangeDetection"="false"),
+  ("format" = "parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-heterogeneous/parquet-heterogeneous.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-heterogeneous/parquet-heterogeneous.05.query.sqlpp
new file mode 100644
index 0000000..3d26c18
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-heterogeneous/parquet-heterogeneous.05.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT c.*
+FROM TestDataset c
+ORDER BY c.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.01.ddl.sqlpp
new file mode 100644
index 0000000..2155b4e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.01.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test if exists;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE ColumnType1 AS {
+    id: int
+    };
+
+CREATE DATASET ParitionParquetDataset(ColumnType1)
+PRIMARY KEY id WITH {
+    "storage-format": {"format" : "column"}
+};
+
+
+CREATE TYPE ColumnType2 AS {
+    };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.02.update.sqlpp
new file mode 100644
index 0000000..4de223e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.02.update.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+LOAD DATASET ParitionParquetDataset USING localfs
+(
+    ("path" = "asterix_nc1://data/hdfs/parquet/partition_heterogeneous.json"),
+    ("format" = "json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.03.update.sqlpp
new file mode 100644
index 0000000..fd2053e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.03.update.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+SELECT t.* FROM ParitionParquetDataset t
+    ) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-partition-heterogeneous" , partitioner_key , "random" )
+OVER ( PARTITION BY toWriter.partitioner_key AS partitioner_key )
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"parquet",
+    "max-schemas" : "10"
+    };
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.04.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.04.ddl.sqlpp
new file mode 100644
index 0000000..6e3e17b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.04.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+
+
+CREATE EXTERNAL DATASET ParitionParquetDatasetCopy(ColumnType2) USING %adapter%
+(
+  %template%,
+  %additional_Properties%,
+  ("definition"="%path_prefix%copy-to-result/parquet-partition-heterogeneous/"),
+  ("include"="*.parquet"),
+  ("requireVersionChangeDetection"="false"),
+  ("format" = "parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.05.query.sqlpp
new file mode 100644
index 0000000..7009f98
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.05.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT c.*
+FROM ParitionParquetDatasetCopy c
+ORDER BY c.id;
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-simple/parquet-simple.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-simple/parquet-simple.01.ddl.sqlpp
new file mode 100644
index 0000000..76970a5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-simple/parquet-simple.01.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test if exists;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE ColumnType2 AS {
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-simple/parquet-simple.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-simple/parquet-simple.02.update.sqlpp
new file mode 100644
index 0000000..9b21be7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-simple/parquet-simple.02.update.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+COPY (
+   select "123" as id
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-simple")
+TYPE ( {id:string} )
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"parquet",
+    "version" : "2"
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-simple/parquet-simple.03.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-simple/parquet-simple.03.ddl.sqlpp
new file mode 100644
index 0000000..f1a4798
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-simple/parquet-simple.03.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+CREATE EXTERNAL DATASET DatasetCopy(ColumnType2) USING %adapter%
+(
+  %template%,
+  %additional_Properties%,
+  ("definition"="%path_prefix%copy-to-result/parquet-simple"),
+  ("format" = "parquet"),
+  ("requireVersionChangeDetection"="false"),
+  ("include"="*.parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-simple/parquet-simple.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-simple/parquet-simple.04.query.sqlpp
new file mode 100644
index 0000000..5aeedb8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-simple/parquet-simple.04.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT id
+FROM DatasetCopy c
+ORDER BY c.id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-tweet/parquet-tweet.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-tweet/parquet-tweet.01.ddl.sqlpp
new file mode 100644
index 0000000..f890e0d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-tweet/parquet-tweet.01.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test if exists;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE ColumnType1 AS {
+    id: string
+};
+
+CREATE DATASET DummyTweetDataset(ColumnType1)
+PRIMARY KEY id WITH {
+    "storage-format": {"format" : "column"}
+};
+
+
+CREATE TYPE ColumnType2 AS {
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-tweet/parquet-tweet.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-tweet/parquet-tweet.02.update.sqlpp
new file mode 100644
index 0000000..83a1140
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-tweet/parquet-tweet.02.update.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+LOAD DATASET DummyTweetDataset USING localfs
+(
+    ("path" = "asterix_nc1://data/hdfs/parquet/dummy_tweet.json"),
+    ("format" = "json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-tweet/parquet-tweet.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-tweet/parquet-tweet.03.update.sqlpp
new file mode 100644
index 0000000..28872dd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-tweet/parquet-tweet.03.update.sqlpp
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+   SELECT c.* FROM DummyTweetDataset c
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-tweet")
+TYPE ( {
+               coordinates: {
+                 coordinates: [
+                   double
+                 ],
+                 `type` : string
+               },
+               created_at: string,
+               entities: {
+                 urls: [
+                   {
+                     display_url: string,
+                     expanded_url: string,
+                     indices: [
+                       int
+                     ],
+                     url: string
+                   }
+                 ],
+                 user_mentions: [
+                   {
+                     id: int,
+                     id_str: string,
+                     indices: [
+                       int
+                     ],
+                     name: string,
+                     screen_name: string
+                   }
+                 ]
+               },
+               favorite_count: int,
+               favorited: boolean,
+               filter_level: string,
+               geo: {
+                 coordinates: [
+                   double
+                 ],
+                 `type`: string
+               },
+               id: string,
+               id_str: string,
+               in_reply_to_screen_name: string,
+               in_reply_to_status_id: int,
+               in_reply_to_status_id_str: string,
+               in_reply_to_user_id: int,
+               in_reply_to_user_id_str: string,
+               is_quote_status: boolean,
+               lang: string,
+               place: {
+                 bounding_box: {
+                   coordinates: [
+                     [
+                       [
+                         double
+                       ]
+                     ]
+                   ],
+                   `type`: string
+                 },
+                 country: string,
+                 country_code: string,
+                 full_name: string,
+                 id: string,
+                 name: string,
+                 place_type: string,
+                 url: string
+               },
+               possibly_sensitive: boolean,
+               quoted_status: {
+                 created_at: string,
+                 entities: {
+                   user_mentions: [
+                     {
+                       id: int,
+                       id_str: string,
+                       indices: [
+                         int
+                       ],
+                       name: string,
+                       screen_name: string
+                     }
+                   ]
+                 },
+                 favorite_count: int,
+                 favorited: boolean,
+                 filter_level: string,
+                 id: int,
+                 id_str: string,
+                 in_reply_to_screen_name: string,
+                 in_reply_to_status_id: int,
+                 in_reply_to_status_id_str: string,
+                 in_reply_to_user_id: int,
+                 in_reply_to_user_id_str: string,
+                 is_quote_status: boolean,
+                 lang: string,
+                 retweet_count: int,
+                 retweeted: boolean,
+                 source: string,
+                 text: string,
+                 truncated: boolean,
+                 user: {
+                   contributors_enabled: boolean,
+                   created_at: string,
+                   default_profile: boolean,
+                   default_profile_image: boolean,
+                   description: string,
+                   favourites_count: int,
+                   followers_count: int,
+                   friends_count: int,
+                   geo_enabled: boolean,
+                   id: int,
+                   id_str: string,
+                   is_translator: boolean,
+                   lang: string,
+                   listed_count: int,
+                   name: string,
+                   profile_background_color: string,
+                   profile_background_image_url: string,
+                   profile_background_image_url_https: string,
+                   profile_background_tile: boolean,
+                   profile_banner_url: string,
+                   profile_image_url: string,
+                   profile_image_url_https: string,
+                   profile_link_color: string,
+                   profile_sidebar_border_color: string,
+                   profile_sidebar_fill_color: string,
+                   profile_text_color: string,
+                   profile_use_background_image: boolean,
+                   protected: boolean,
+                   screen_name: string,
+                   statuses_count: int,
+                   verified: boolean
+                 }
+               },
+               quoted_status_id: int,
+               quoted_status_id_str: string,
+               retweet_count: int,
+               retweeted: boolean,
+               source: string,
+               text: string,
+               timestamp_ms: string,
+               truncated: boolean,
+               user: {
+                 contributors_enabled: boolean,
+                 created_at: string,
+                 default_profile: boolean,
+                 default_profile_image: boolean,
+                 description: string,
+                 favourites_count: int,
+                 followers_count: int,
+                 friends_count: int,
+                 geo_enabled: boolean,
+                 id: int,
+                 id_str: string,
+                 is_translator: boolean,
+                 lang: string,
+                 listed_count: int,
+                 location: string,
+                 name: string,
+                 profile_background_color: string,
+                 profile_background_image_url: string,
+                 profile_background_image_url_https: string,
+                 profile_background_tile: boolean,
+                 profile_banner_url: string,
+                 profile_image_url: string,
+                 profile_image_url_https: string,
+                 profile_link_color: string,
+                 profile_sidebar_border_color: string,
+                 profile_sidebar_fill_color: string,
+                 profile_text_color: string,
+                 profile_use_background_image: boolean,
+                 protected: boolean,
+                 screen_name: string,
+                 statuses_count: int,
+                 time_zone: string,
+                 url: string,
+                 utc_offset: int,
+                 verified: boolean
+               }
+             } )
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"parquet"
+};
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-tweet/parquet-tweet.04.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-tweet/parquet-tweet.04.ddl.sqlpp
new file mode 100644
index 0000000..5a93999
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-tweet/parquet-tweet.04.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+
+
+CREATE EXTERNAL DATASET DummyTweetDatasetCopy(ColumnType2) USING %adapter%
+(
+  %template%,
+  %additional_Properties%,
+  ("definition"="%path_prefix%copy-to-result/parquet-tweet/"),
+  ("include"="*.parquet"),
+  ("requireVersionChangeDetection"="false"),
+  ("format" = "parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-tweet/parquet-tweet.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-tweet/parquet-tweet.05.query.sqlpp
new file mode 100644
index 0000000..13587f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-tweet/parquet-tweet.05.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT c.*
+FROM DummyTweetDatasetCopy c
+ORDER BY c.id;
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-utf8/parquet-utf8.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-utf8/parquet-utf8.01.ddl.sqlpp
new file mode 100644
index 0000000..dfc64ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-utf8/parquet-utf8.01.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test if exists;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE ColumnType1 AS {
+    id: int
+};
+
+CREATE DATASET NameCommentDataset(ColumnType1)
+PRIMARY KEY id WITH {
+    "storage-format": {"format" : "column"}
+};
+
+
+CREATE TYPE ColumnType2 AS {
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-utf8/parquet-utf8.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-utf8/parquet-utf8.02.update.sqlpp
new file mode 100644
index 0000000..8591369
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-utf8/parquet-utf8.02.update.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+LOAD DATASET NameCommentDataset USING localfs
+(
+    ("path" = "asterix_nc1://data/hdfs/parquet/id_name_comment.json"),
+    ("format" = "json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-utf8/parquet-utf8.03.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-utf8/parquet-utf8.03.update.sqlpp
new file mode 100644
index 0000000..ad94373
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-utf8/parquet-utf8.03.update.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+COPY (
+   SELECT c.* FROM NameCommentDataset c
+) toWriter
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "parquet-utf8")
+TYPE ( { comment:string, id:bigint, name:string } )
+WITH {
+    %template_colons%,
+    %additionalProperties%
+    "format":"parquet"
+};
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-utf8/parquet-utf8.04.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-utf8/parquet-utf8.04.ddl.sqlpp
new file mode 100644
index 0000000..bf71240
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-utf8/parquet-utf8.04.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+
+
+CREATE EXTERNAL DATASET NameCommentDatasetCopy(ColumnType2) USING %adapter%
+(
+  %template%,
+  %additional_Properties%,
+  ("definition"="%path_prefix%copy-to-result/parquet-utf8/"),
+  ("include"="*.parquet"),
+  ("requireVersionChangeDetection"="false"),
+  ("format" = "parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-utf8/parquet-utf8.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-utf8/parquet-utf8.05.query.sqlpp
new file mode 100644
index 0000000..17cd027
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/parquet-utf8/parquet-utf8.05.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+
+SELECT c.*
+FROM NameCommentDatasetCopy c
+ORDER BY c.id;
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/partition/partition.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/partition/partition.01.ddl.sqlpp
index b68c38b..a384802 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/partition/partition.01.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/partition/partition.01.ddl.sqlpp
@@ -24,13 +24,10 @@
 CREATE TYPE OpenType AS {
 };
 
-CREATE EXTERNAL DATASET Customer(OpenType) USING S3 (
-    ("accessKeyId"="dummyAccessKey"),
-    ("secretAccessKey"="dummySecretKey"),
-    ("region"="us-west-2"),
-    ("serviceEndpoint"="http://127.0.0.1:8001"),
-    ("container"="playground"),
-    ("definition"="external-filter/car/{company:string}/customer/{customer_id:int}"),
+CREATE EXTERNAL DATASET Customer(OpenType) USING %adapter% (
+    %template%,
+    %additional_Properties%,
+    ("definition"="%path_prefix%external-filter/car/{company:string}/customer/{customer_id:int}"),
     ("embed-filter-values" = "false"),
     ("format"="json")
 );
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/partition/partition.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/partition/partition.02.update.sqlpp
index f1a22d0..a9a7a8f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/partition/partition.02.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/partition/partition.02.update.sqlpp
@@ -20,18 +20,15 @@
 USE test;
 
 COPY Customer c
-TO S3
-PATH ("copy-to-result", "car", company, "customer", customer_id)
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "car", company, "customer", customer_id)
 OVER (
    PARTITION BY c.company company,
                 c.customer_id customer_id
 )
 WITH {
-    "accessKeyId":"dummyAccessKey",
-    "secretAccessKey":"dummySecretKey",
-    "region":"us-west-2",
-    "serviceEndpoint":"http://127.0.0.1:8001",
-    "container":"playground",
+    %template_colons%,
+    %additionalProperties%
     "format":"json",
     "compression":"gzip"
 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/partition/partition.03.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/partition/partition.03.ddl.sqlpp
index 14d1d92..f46ddf9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/partition/partition.03.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/partition/partition.03.ddl.sqlpp
@@ -19,13 +19,10 @@
 
 USE test;
 
-CREATE EXTERNAL DATASET CustomerCopy(OpenType) USING S3 (
-    ("accessKeyId"="dummyAccessKey"),
-    ("secretAccessKey"="dummySecretKey"),
-    ("region"="us-west-2"),
-    ("serviceEndpoint"="http://127.0.0.1:8001"),
-    ("container"="playground"),
-    ("definition"="copy-to-result/car/{company:string}/customer/{customer_id:int}"),
+CREATE EXTERNAL DATASET CustomerCopy(OpenType) USING %adapter% (
+    %template%,
+    %additional_Properties%,
+    ("definition"="%path_prefix%copy-to-result/car/{company:string}/customer/{customer_id:int}"),
     ("embed-filter-values" = "false"),
     ("format"="json")
 );
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/simple-write/simple-write.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/simple-write/simple-write.01.ddl.sqlpp
index b68c38b..a384802 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/simple-write/simple-write.01.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/simple-write/simple-write.01.ddl.sqlpp
@@ -24,13 +24,10 @@
 CREATE TYPE OpenType AS {
 };
 
-CREATE EXTERNAL DATASET Customer(OpenType) USING S3 (
-    ("accessKeyId"="dummyAccessKey"),
-    ("secretAccessKey"="dummySecretKey"),
-    ("region"="us-west-2"),
-    ("serviceEndpoint"="http://127.0.0.1:8001"),
-    ("container"="playground"),
-    ("definition"="external-filter/car/{company:string}/customer/{customer_id:int}"),
+CREATE EXTERNAL DATASET Customer(OpenType) USING %adapter% (
+    %template%,
+    %additional_Properties%,
+    ("definition"="%path_prefix%external-filter/car/{company:string}/customer/{customer_id:int}"),
     ("embed-filter-values" = "false"),
     ("format"="json")
 );
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/simple-write/simple-write.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/simple-write/simple-write.02.update.sqlpp
index 94aa7a0..0ee6d29 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/simple-write/simple-write.02.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/simple-write/simple-write.02.update.sqlpp
@@ -20,14 +20,11 @@
 USE test;
 
 COPY Customer c
-TO S3
-PATH ("copy-to-result", "simple-write")
+TO %adapter%
+PATH (%pathprefix% "copy-to-result", "simple-write")
 WITH {
-    "accessKeyId":"dummyAccessKey",
-    "secretAccessKey":"dummySecretKey",
-    "region":"us-west-2",
-    "serviceEndpoint":"http://127.0.0.1:8001",
-    "container":"playground",
+    %template_colons%,
+    %additionalProperties%
     "format":"json"
 }
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/simple-write/simple-write.03.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/simple-write/simple-write.03.ddl.sqlpp
index 5f03abd..1c34766 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/simple-write/simple-write.03.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/copy-to/simple-write/simple-write.03.ddl.sqlpp
@@ -19,13 +19,10 @@
 
 USE test;
 
-CREATE EXTERNAL DATASET CustomerCopy(OpenType) USING S3 (
-    ("accessKeyId"="dummyAccessKey"),
-    ("secretAccessKey"="dummySecretKey"),
-    ("region"="us-west-2"),
-    ("serviceEndpoint"="http://127.0.0.1:8001"),
-    ("container"="playground"),
-    ("definition"="copy-to-result/simple-write"),
+CREATE EXTERNAL DATASET CustomerCopy(OpenType) USING %adapter% (
+    %template%,
+    %additional_Properties%,
+    ("definition"="%path_prefix%copy-to-result/simple-write"),
     ("embed-filter-values" = "false"),
     ("format"="json")
 );
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/array-access/array-access.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/array-access/array-access.01.ddl.sqlpp
index 606c781..76ae449 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/array-access/array-access.01.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/array-access/array-access.01.ddl.sqlpp
@@ -34,8 +34,8 @@
 CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
 (
   %template%,
-  ("container"="playground"),
-  ("definition"="avro-data/reviews"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%avro-data/reviews"),
   ("include"="*dummy_tweet.avro"),
   ("format" = "avro")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-map/avro-map.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-map/avro-map.01.ddl.sqlpp
index 65a2b38..ca20804 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-map/avro-map.01.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-map/avro-map.01.ddl.sqlpp
@@ -34,8 +34,8 @@
 CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
 (
   %template%,
-  ("container"="playground"),
-  ("definition"="avro-data/reviews"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%avro-data/reviews"),
   ("include"="*avro_type.avro"),
   ("format" = "avro")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-nested-records/avro-nested-records.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-nested-records/avro-nested-records.01.ddl.sqlpp
index 65a2b38..ca20804 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-nested-records/avro-nested-records.01.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-nested-records/avro-nested-records.01.ddl.sqlpp
@@ -34,8 +34,8 @@
 CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
 (
   %template%,
-  ("container"="playground"),
-  ("definition"="avro-data/reviews"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%avro-data/reviews"),
   ("include"="*avro_type.avro"),
   ("format" = "avro")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-primitives/avro-primitives.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-primitives/avro-primitives.01.ddl.sqlpp
index 65a2b38..ca20804 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-primitives/avro-primitives.01.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-primitives/avro-primitives.01.ddl.sqlpp
@@ -34,8 +34,8 @@
 CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
 (
   %template%,
-  ("container"="playground"),
-  ("definition"="avro-data/reviews"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%avro-data/reviews"),
   ("include"="*avro_type.avro"),
   ("format" = "avro")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-union/avro-union.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-union/avro-union.01.ddl.sqlpp
index 8c00ded..1b97979 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-union/avro-union.01.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-union/avro-union.01.ddl.sqlpp
@@ -34,8 +34,8 @@
 CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
 (
   %template%,
-  ("container"="playground"),
-  ("definition"="avro-data/reviews"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%avro-data/reviews"),
   ("include"="*avro_type.avro"),
   ("format" = "avro")
 );
@@ -43,8 +43,9 @@
 CREATE EXTERNAL DATASET AvroDataset2(AvroType) USING %adapter%
 (
   %template%,
+  %additional_Properties%,
   ("container"="playground"),
-  ("definition"="avro-data/reviews"),
+  ("definition"="%path_prefix%avro-data/reviews"),
   ("include"="*partition_heterogeneous.avro"),
   ("format" = "avro")
 );
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/field-access/field-access.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/field-access/field-access.01.ddl.sqlpp
index 5e30b26..74b7f17 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/field-access/field-access.01.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/field-access/field-access.01.ddl.sqlpp
@@ -34,8 +34,8 @@
 CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
 (
   %template%,
-  ("container"="playground"),
-  ("definition"="avro-data/reviews"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%avro-data/reviews"),
   ("include"="*id_age.avro"),
   ("format" = "avro")
 );
@@ -43,8 +43,8 @@
 CREATE EXTERNAL DATASET AvroDataset2(AvroType) USING %adapter%
 (
   %template%,
-  ("container"="playground"),
-  ("definition"="avro-data/reviews"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%avro-data/reviews"),
   ("include"="*id_age.avro"),
   ("format" = "avro")
 );
@@ -52,8 +52,8 @@
 CREATE EXTERNAL DATASET AvroDataset3(AvroType) USING %adapter%
 (
   %template%,
-  ("container"="playground"),
-  ("definition"="avro-data/reviews"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%avro-data/reviews"),
   ("include"="*id_name_comment.avro"),
   ("format" = "avro")
 );
@@ -61,8 +61,8 @@
 CREATE EXTERNAL DATASET AvroDataset4(AvroType) USING %adapter%
 (
   %template%,
-  ("container"="playground"),
-  ("definition"="avro-data/reviews"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%avro-data/reviews"),
   ("include"="*dummy_tweet.avro"),
   ("format" = "avro")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/heterogeneous-access/heterogeneous-access.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/heterogeneous-access/heterogeneous-access.1.ddl.sqlpp
index c7dcf7a..0ea0cc2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/heterogeneous-access/heterogeneous-access.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/heterogeneous-access/heterogeneous-access.1.ddl.sqlpp
@@ -34,8 +34,8 @@
 CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
 (
   %template%,
-  ("container"="playground"),
-  ("definition"="avro-data/reviews"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%avro-data/reviews"),
   ("include"="*heterogeneous_*"),
   ("format" = "avro")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/invalid-avro-files/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/invalid-avro-files/test.000.ddl.sqlpp
index a3a6d0d..ac678fb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/invalid-avro-files/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/invalid-avro-files/test.000.ddl.sqlpp
@@ -28,7 +28,7 @@
 CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
 (
   %template%,
-  ("container"="playground"),
-  ("definition"="json-data/reviews/single-line/json"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%json-data/reviews/single-line/json"),
   ("format" = "avro")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/invalid-type/invalid-type.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/invalid-type/invalid-type.1.ddl.sqlpp
index c6ee970..4f4e322 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/invalid-type/invalid-type.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/invalid-type/invalid-type.1.ddl.sqlpp
@@ -35,8 +35,8 @@
 CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
 (
   %template%,
-  ("container"="playground"),
-  ("definition"="avro-data/reviews"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%avro-data/reviews"),
   ("include"="*id_age.avro"),
   ("format" = "avro")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/missing-fields/missing-fields.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/missing-fields/missing-fields.1.ddl.sqlpp
index 3c8e934..74664c5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/missing-fields/missing-fields.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/missing-fields/missing-fields.1.ddl.sqlpp
@@ -34,8 +34,8 @@
 CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
 (
   %template%,
-  ("container"="playground"),
-  ("definition"="avro-data/reviews"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%avro-data/reviews"),
   ("include"="*dummy_tweet.avro"),
   ("format" = "avro")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/multi-file-multi-schema/multi-file-multi-schema.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/multi-file-multi-schema/multi-file-multi-schema.1.ddl.sqlpp
index 67e38d2..92dca50 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/multi-file-multi-schema/multi-file-multi-schema.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/multi-file-multi-schema/multi-file-multi-schema.1.ddl.sqlpp
@@ -34,8 +34,8 @@
 CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
 (
   %template%,
-  ("container"="playground"),
-  ("definition"="avro-data/reviews"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%avro-data/reviews"),
   ("include"="*id_age.avro"),
   ("include#1"="*id_name.avro"),
   ("format" = "avro")
@@ -44,8 +44,8 @@
 CREATE EXTERNAL DATASET AvroDataset2(AvroType) USING %adapter%
 (
   %template%,
-  ("container"="playground"),
-  ("definition"="avro-data/reviews"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%avro-data/reviews"),
   ("include"="*id_age.avro"),
   ("include#1"="*id_age-string.avro"),
   ("format" = "avro")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/no-files/no-files.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/no-files/no-files.1.ddl.sqlpp
index 507d6fa..950bfba 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/no-files/no-files.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/no-files/no-files.1.ddl.sqlpp
@@ -36,7 +36,7 @@
 CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
 (
   %template%,
-  ("container"="playground"),
+  %additional_Properties%,
   ("definition"="NOT_A_DEFINITION"),
   ("format" = "avro")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/object-concat/object-concat.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/object-concat/object-concat.1.ddl.sqlpp
index c7f127b..6f64c80 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/object-concat/object-concat.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/object-concat/object-concat.1.ddl.sqlpp
@@ -34,8 +34,8 @@
 CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
 (
   %template%,
-  ("container"="playground"),
-  ("definition"="avro-data/reviews"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%avro-data/reviews"),
   ("include"="*dummy_tweet.avro"),
   ("format" = "avro")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/select-all-fields/select-all-fields.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/select-all-fields/select-all-fields.1.ddl.sqlpp
index 6d77dab..fb2d3db 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/select-all-fields/select-all-fields.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/select-all-fields/select-all-fields.1.ddl.sqlpp
@@ -34,8 +34,8 @@
 CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
 (
 %template%,
-("container"="playground"),
-("definition"="avro-data/reviews"),
+%additional_Properties%,
+("definition"="%path_prefix%avro-data/reviews"),
 ("include"="*dummy_tweet.avro"),
 ("format" = "avro")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/select-count-one-field/select-count-one-field.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/select-count-one-field/select-count-one-field.1.ddl.sqlpp
index 052b127..285d19c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/select-count-one-field/select-count-one-field.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/select-count-one-field/select-count-one-field.1.ddl.sqlpp
@@ -34,8 +34,8 @@
 CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
 (
   %template%,
-  ("container"="playground"),
-  ("definition"="avro-data/reviews"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%avro-data/reviews"),
   ("include"="*dummy_tweet.avro"),
   ("format" = "avro")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/string-standard-utf8/string-standard-utf8.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/string-standard-utf8/string-standard-utf8.1.ddl.sqlpp
index 17433a5..d36da08 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/string-standard-utf8/string-standard-utf8.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/string-standard-utf8/string-standard-utf8.1.ddl.sqlpp
@@ -34,8 +34,8 @@
 CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
 (
   %template%,
-  ("container"="playground"),
-  ("definition"="avro-data/reviews"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%avro-data/reviews"),
   ("include"="*id_name_comment.avro"),
   ("format" = "avro")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/type-mismatch/type-mismatch.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/type-mismatch/type-mismatch.1.ddl.sqlpp
index c8d7d21..d6d9dec 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/type-mismatch/type-mismatch.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/type-mismatch/type-mismatch.1.ddl.sqlpp
@@ -34,8 +34,8 @@
 CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
 (
   %template%,
-  ("container"="playground"),
-  ("definition"="avro-data/reviews"),
+  %additional_Properties%,
+  ("definition"="%path_prefix%avro-data/reviews"),
   ("include"="*dummy_tweet.avro"),
   ("format" = "avro")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/deltalake-file-nine/deltalake-file-nine.00.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/deltalake-file-nine/deltalake-file-nine.00.ddl.sqlpp
new file mode 100644
index 0000000..c1a74c5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/deltalake-file-nine/deltalake-file-nine.00.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+ DROP DATAVERSE test IF EXISTS;
+ CREATE DATAVERSE test;
+
+ USE test;
+
+
+ CREATE TYPE DeltalakeTableType as {
+ };
+
+ CREATE EXTERNAL COLLECTION DeltalakeDataset(DeltalakeTableType) USING %adapter%
+ (
+   %template%,
+   ("container"="playground"),
+   ("definition"="delta-data/delta_file_size_nine"),
+   ("table-format" = "delta")
+ );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/deltalake-file-nine/deltalake-file-nine.01.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/deltalake-file-nine/deltalake-file-nine.01.query.sqlpp
new file mode 100644
index 0000000..db2abf5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/deltalake-file-nine/deltalake-file-nine.01.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+ USE test;
+
+ SELECT element ds FROM DeltalakeDataset as ds order by id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/deltalake-file-one/deltalake-file-one.00.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/deltalake-file-one/deltalake-file-one.00.ddl.sqlpp
new file mode 100644
index 0000000..1284e93
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/deltalake-file-one/deltalake-file-one.00.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+ DROP DATAVERSE test IF EXISTS;
+ CREATE DATAVERSE test;
+
+ USE test;
+
+
+ CREATE TYPE DeltalakeTableType as {
+ };
+
+ CREATE EXTERNAL COLLECTION DeltalakeDataset(DeltalakeTableType) USING %adapter%
+ (
+   %template%,
+   ("container"="playground"),
+   ("definition"="delta-data/delta_file_size_one"),
+   ("table-format" = "delta")
+ );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/deltalake-file-one/deltalake-file-one.01.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/deltalake-file-one/deltalake-file-one.01.query.sqlpp
new file mode 100644
index 0000000..84e7914
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/deltalake-file-one/deltalake-file-one.01.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+ USE test;
+
+ SELECT element ds FROM DeltalakeDataset as ds;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/embed-flat/embed-flat.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/embed-flat/embed-flat.000.ddl.sqlpp
index 654a84a..dd5c56d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/embed-flat/embed-flat.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/embed-flat/embed-flat.000.ddl.sqlpp
@@ -30,8 +30,8 @@
 
 CREATE EXTERNAL DATASET Name(TestType) USING %adapter% (
     %template%,
-    ("container"="playground"),
-    ("definition"="avro-data/external-filter/embed/name/{name:string}"),
+    %additional_Properties%,
+    ("definition"="%path_prefix%avro-data/external-filter/embed/name/{name:string}"),
     ("embed-filter-values" = "true"),
     ("format"="avro")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/embed-multiple-values/embed-multiple-values.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/embed-multiple-values/embed-multiple-values.000.ddl.sqlpp
index be3d97f..b835823 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/embed-multiple-values/embed-multiple-values.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/embed-multiple-values/embed-multiple-values.000.ddl.sqlpp
@@ -26,8 +26,8 @@
 
 CREATE EXTERNAL DATASET maintenance(test) USING %adapter% (
     %template%,
-    ("container"="playground"),
-    ("definition"="avro-data/external-filter/embed/car/{company:string}/customer/{customer_id:int}/maintenance-report/year-{year:int}-month-{month:int}-day-{day:int}-date"),
+    %additional_Properties%,
+    ("definition"="%path_prefix%avro-data/external-filter/embed/car/{company:string}/customer/{customer_id:int}/maintenance-report/year-{year:int}-month-{month:int}-day-{day:int}-date"),
     ("embed-filter-values" = "true"),
     ("format"="avro")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/embed-one-value/embed-one-value.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/embed-one-value/embed-one-value.000.ddl.sqlpp
index b1b1b76..c947f70 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/embed-one-value/embed-one-value.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/embed-one-value/embed-one-value.000.ddl.sqlpp
@@ -27,16 +27,16 @@
 
 CREATE EXTERNAL DATASET Department(TestType) USING %adapter% (
     %template%,
-    ("container"="playground"),
-    ("definition"="avro-data/external-filter/embed/department/{department:string}"),
+    %additional_Properties%,
+    ("definition"="%path_prefix%avro-data/external-filter/embed/department/{department:string}"),
     ("embed-filter-values" = "true"),
     ("format"="avro")
 );
 
 CREATE EXTERNAL DATASET LastName(TestType) USING %adapter% (
     %template%,
-    ("container"="playground"),
-    ("definition"="avro-data/external-filter/embed/last-name/{name.last:string}"),
+    %additional_Properties%,
+    ("definition"="%path_prefix%avro-data/external-filter/embed/last-name/{name.last:string}"),
     ("embed-filter-values" = "true"),
     ("format"="avro")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/embed-with-nested-values/embed-flat.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/embed-with-nested-values/embed-flat.000.ddl.sqlpp
index 2c5b352..531e398 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/embed-with-nested-values/embed-flat.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/embed-with-nested-values/embed-flat.000.ddl.sqlpp
@@ -30,8 +30,8 @@
 
 CREATE EXTERNAL DATASET Name(TestType) USING %adapter% (
     %template%,
-    ("container"="playground"),
-    ("definition"="avro-data/external-filter/embed/nested-value/{name:string}"),
+    %additional_Properties%,
+    ("definition"="%path_prefix%avro-data/external-filter/embed/nested-value/{name:string}"),
     ("embed-filter-values" = "true"),
     ("format"="avro")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.000.ddl.sqlpp
index 5fcfff5..173d1d0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.000.ddl.sqlpp
@@ -27,16 +27,16 @@
 
 CREATE EXTERNAL DATASET Department(TestType) USING %adapter% (
     %template%,
-    ("container"="playground"),
-    ("definition"="avro-data/external-filter/department/{department:string}"),
+    %additional_Properties%,
+    ("definition"="%path_prefix%avro-data/external-filter/department/{department:string}"),
     ("embed-filter-values" = "false"),
     ("format"="avro")
 );
 
 CREATE EXTERNAL DATASET LastName(TestType) USING %adapter% (
     %template%,
-    ("container"="playground"),
-    ("definition"="avro-data/external-filter/last-name/{name.last:string}"),
+    %additional_Properties%,
+    ("definition"="%path_prefix%avro-data/external-filter/last-name/{name.last:string}"),
     ("embed-filter-values" = "false"),
     ("format"="avro")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/query/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/query/test.000.ddl.sqlpp
index 4c418b4..49cc3e6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/query/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/query/test.000.ddl.sqlpp
@@ -27,24 +27,24 @@
 
 CREATE EXTERNAL DATASET company(test) USING %adapter% (
     %template%,
-    ("container"="playground"),
-    ("definition"="avro-data/external-filter/car/{company:string}"),
+    %additional_Properties%,
+    ("definition"="%path_prefix%avro-data/external-filter/car/{company:string}"),
     ("embed-filter-values" = "false"),
     ("format"="avro")
 );
 
 CREATE EXTERNAL DATASET customer(test) USING %adapter% (
     %template%,
-    ("container"="playground"),
-    ("definition"="avro-data/external-filter/car/{company:string}/customer/{customer_id:int}"),
+    %additional_Properties%,
+    ("definition"="%path_prefix%avro-data/external-filter/car/{company:string}/customer/{customer_id:int}"),
     ("embed-filter-values" = "false"),
     ("format"="avro")
 );
 
 CREATE EXTERNAL DATASET maintenance(test) USING %adapter% (
     %template%,
-    ("container"="playground"),
-    ("definition"="avro-data/external-filter/car/{company:string}/customer/{customer_id:int}/maintenance-report/year-{year:int}-month-{month:int}-day-{day:int}-date"),
+    %additional_Properties%,
+    ("definition"="%path_prefix%avro-data/external-filter/car/{company:string}/customer/{customer_id:int}/maintenance-report/year-{year:int}-month-{month:int}-day-{day:int}-date"),
     ("embed-filter-values" = "false"),
     ("format"="avro")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/using-limit/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/using-limit/test.000.ddl.sqlpp
index 13316e8..073965b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/using-limit/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/using-limit/test.000.ddl.sqlpp
@@ -26,8 +26,8 @@
 
 CREATE EXTERNAL DATASET maintenance(test) USING %adapter% (
     %template%,
-    ("container"="playground"),
-    ("definition"="avro-data/external-filter/car/{company:string}/customer/{customer_id:int}/maintenance-report/year-{year:int}-month-{month:int}-day-{day:int}-date"),
+    %additional_Properties%,
+    ("definition"="%path_prefix%avro-data/external-filter/car/{company:string}/customer/{customer_id:int}/maintenance-report/year-{year:int}-month-{month:int}-day-{day:int}-date"),
     ("embed-filter-values" = "false"),
     ("format"="avro")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/views/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/views/test.000.ddl.sqlpp
index 3c2b513..495acc8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/views/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/dynamic-prefixes/avro/views/test.000.ddl.sqlpp
@@ -26,8 +26,8 @@
 
 CREATE EXTERNAL DATASET maintenance(test) USING %adapter% (
     %template%,
-    ("container"="playground"),
-    ("definition"="avro-data/external-filter/embed/car/{company:string}/customer/{customer_id:int}/maintenance-report/year-{year:int}-month-{month:int}-day-{day:int}-date"),
+    %additional_Properties%,
+    ("definition"="%path_prefix%avro-data/external-filter/embed/car/{company:string}/customer/{customer_id:int}/maintenance-report/year-{year:int}-month-{month:int}-day-{day:int}-date"),
     ("embed-filter-values" = "true"),
     ("format"="avro")
 );
@@ -39,8 +39,8 @@
 
 CREATE EXTERNAL DATASET maintenance2(test) USING %adapter% (
     %template%,
-    ("container"="playground"),
-    ("definition"="avro-data/external-filter/embed/car/{company:string}/customer/{customerId:int}/maintenance-report/year-{year:int}-month-{month:int}-day-{day:int}-date"),
+    %additional_Properties%,
+    ("definition"="%path_prefix%avro-data/external-filter/embed/car/{company:string}/customer/{customerId:int}/maintenance-report/year-{year:int}-month-{month:int}-day-{day:int}-date"),
     ("embed-filter-values" = "true"),
     ("format"="avro")
 );
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.02.update.sqlpp
index 3ab3bc6..6bc1564 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.02.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.02.update.sqlpp
@@ -27,7 +27,13 @@
 {"id": 129, "myGeometry": st_geom_from_geojson({"type": "MultiPolygon","coordinates": [[[[40, 40], [20, 45], [45, 30], [40, 40]]],[[[20, 35], [10, 30], [10, 10], [30, 5], [45, 20], [20, 35]],[[30, 20], [20, 15], [20, 25], [30, 20]]]]})},
 {"id": 130, "myGeometry": st_make_point(-71.1043443253471, 42.3150676015829)},
 {"id": 131, "myGeometry": st_make_point(1.0,2.0,3.0)},
-{"id": 132, "myGeometry": st_make_point(1.0,2.0,3.0,4.0)},
+/* The query below is intentionally commented as JTS WKBWriter does not support 4 dimensional coordinates https://github.com/locationtech/jts/issues/733, If uncommented add the below assertions to the files mentioned
+line 9 at result.03.adm should have the result { "Geometries": { "id": 132, "myGeometry": {"type":"Point","coordinates":[1,2,3,4]} } }
+line 9 at result.04.adm should have the result {{ "Type": "Point", "Area": 0.0, "Coordinate dimension": 4, "Dimension": 0, "NPoints": 1, "XMax": 1.0, "XMin": 1.0, "YMax": 2.0, "YMin": 2.0, "Binary": hex("01B90B0000000000000000F03F000000000000004000000000000008400000000000001040"), "GeoJSON": "{\"type\":\"Point\",\"coordinates\":[1,2,3,4]}", "WKT": "POINT ZM (1 2 3 4)" }
+line 4 at result.05.adm should have the result { "X": 1.0, "Y": 2.0, "Z": 3.0, "M": 4.0 }
+line 9 at result.09.adm should have the result { "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+*/
+/* {"id": 132, "myGeometry": st_make_point(1.0,2.0,3.0,4.0)}, */
 {"id": 133, "myGeometry": st_geom_from_text('POLYGON((743238 2967416,743238 2967450,743265 2967450,743265.625 2967416,743238 2967416))')},
 {"id": 134, "myGeometry": st_geom_from_wkb(hex("0102000000020000001F85EB51B87E5CC0D34D621058994340105839B4C87E5CC0295C8FC2F5984340"))},
 {"id": 135, "myGeometry": st_line_from_multipoint(st_geom_from_text('MULTIPOINT(1 2 , 4 5 , 7 8 )'))},
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/index/index.19.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/index/index.19.update.sqlpp
index bdfef62..02d5064 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/index/index.19.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/index/index.19.update.sqlpp
@@ -27,7 +27,11 @@
 {"id": 129, "myGeometry": st_geom_from_geojson({"type": "MultiPolygon","coordinates": [[[[40, 40], [20, 45], [45, 30], [40, 40]]],[[[20, 35], [10, 30], [10, 10], [30, 5], [45, 20], [20, 35]],[[30, 20], [20, 15], [20, 25], [30, 20]]]]})},
 {"id": 130, "myGeometry": st_make_point(-71.1043443253471, 42.3150676015829)},
 {"id": 131, "myGeometry": st_make_point(1.0,2.0,3.0)},
-{"id": 132, "myGeometry": st_make_point(1.0,2.0,3.0,4.0)},
+/* The query below is intentionally commented as JTS WKBWriter does not support 4 dimensional coordinates https://github.com/locationtech/jts/issues/733, If uncommented add the below assertions to the files mentioned
+line 9 at result.20.adm should have the result { "Geometries": { "id": 132, "myGeometry": {"type":"Point","coordinates":[1,2,3,4]} } }
+line 2 at result.21.adm should have the result 132
+*/
+/* {"id": 132, "myGeometry": st_make_point(1.0,2.0,3.0,4.0)}, */
 {"id": 133, "myGeometry": st_geom_from_text('POLYGON((743238 2967416,743238 2967450,743265 2967450,743265.625 2967416,743238 2967416))')},
 {"id": 134, "myGeometry": st_geom_from_wkb(hex("0102000000020000001F85EB51B87E5CC0D34D621058994340105839B4C87E5CC0295C8FC2F5984340"))},
 {"id": 135, "myGeometry": st_line_from_multipoint(st_geom_from_text('MULTIPOINT(1 2 , 4 5 , 7 8 )'))},
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/predicate.15.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/predicate.15.query.sqlpp
new file mode 100644
index 0000000..c43c64e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/predicate.15.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+{
+"a": st_z_max(st_geom_from_text("POLYGON EMPTY")),
+"b": st_z_min(st_geom_from_text("POLYGON EMPTY")),
+"c": st_z_max(st_geom_from_text("POLYGON Z ((0 0 0, 1 1 0, 1 -1 0, 0 0 0))")),
+"d": st_z_min(st_geom_from_text("POLYGON Z ((0 0 0, 1 1 0, 1 -1 0, 0 0 0))")),
+"e": st_z_max(st_geom_from_text("POLYGON Z ((0 0 1, 1 1 2, 1 -1 3, 0 0 1))")),
+"f": st_z_min(st_geom_from_text("POLYGON Z ((0 0 1, 1 1 2, 1 -1 3, 0 0 1))")),
+"g": st_z_max(st_geom_from_text("POLYGON Z ((0 0 -1, 1 1 -2, 1 -1 -3, 0 0 -1))")),
+"h": st_z_min(st_geom_from_text("POLYGON Z ((0 0 -1, 1 1 -2, 1 -1 -3, 0 0 -1))")),
+"i": st_z_max(st_geom_from_text("POLYGON Z ((0 0 -1, 1 1 0, 1 -1 1, 0 0 -1))")),
+"j": st_z_min(st_geom_from_text("POLYGON Z ((0 0 -1, 1 1 0, 1 -1 1, 0 0 -1))")),
+"k": st_z_max(st_geom_from_text("POLYGON Z ((0 0 0, 1 1 0, 1 -1 2, 0 0 0))")),
+"l": st_z_min(st_geom_from_text("POLYGON Z ((0 0 0, 1 1 0, 1 -1 2, 0 0 0))")),
+"m": st_z_max(st_geom_from_text("POLYGON Z ((0 0 0, 1 1 -1, 1 -1 0, 0 0 0))")),
+"n": st_z_min(st_geom_from_text("POLYGON Z ((0 0 0, 1 1 -1, 1 -1 0, 0 0 0))"))
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/secondary-heterogeneous-indexnl-params/secondary-heterogeneous-indexnl-params.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/secondary-heterogeneous-indexnl-params/secondary-heterogeneous-indexnl-params.1.ddl.sqlpp
new file mode 100644
index 0000000..040968e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/secondary-heterogeneous-indexnl-params/secondary-heterogeneous-indexnl-params.1.ddl.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+create type tenkType as {
+  unique1         : integer,
+  unique2         : integer,
+  two             : integer,
+  four            : integer,
+  ten             : integer,
+  twenty          : integer,
+  hundred         : integer,
+  fivethous       : integer,
+  tenthous        : integer,
+  odd100          : integer,
+  even100         : integer,
+  stringu1        : string,
+  stringu2        : string,
+  string4         : string
+};
+
+create dataset tenk1(tenkType) primary key unique2;
+
+create index idx_tenk1_1k on tenk1(thousand);
+
+create dataset tenk2(tenkType) primary key unique2;
+
+create index idx_tenk2_1k on tenk2(thousand);
+
+create index idx_tenk2_1k_2k on tenk2(thousand, twothousand);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/secondary-heterogeneous-indexnl-params/secondary-heterogeneous-indexnl-params.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/secondary-heterogeneous-indexnl-params/secondary-heterogeneous-indexnl-params.2.update.sqlpp
new file mode 100644
index 0000000..c6189ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/secondary-heterogeneous-indexnl-params/secondary-heterogeneous-indexnl-params.2.update.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+
+Upsert INTO tenk1 (
+    [{ "unique1": 5057, "unique2": 6, "two": 1, "four": 1, "ten": 7, "twenty": 17, "hundred": 57, "thousand": 57, "twothousand": 1057, "fivethous": 57, "tenthous": 5057, "odd100": 114, "even100": 115, "stringu1": "NMAAAA", "stringu2": "GAAAAA", "string4": "OOOOxx" }
+, { "unique1": 1891, "unique2": 1, "two": 1, "four": 3, "ten": 1, "twenty": 11, "hundred": 91, "thousand": 891, "twothousand": 1891, "fivethous": 1891, "tenthous": 1891, "odd100": 182, "even100": 183, "stringu1": "TUAAAA", "stringu2": "BAAAAA", "string4": "HHHHxx" }
+, { "unique1": 3420, "unique2": 2, "two": 0, "four": 0, "ten": 0, "twenty": 0, "hundred": 20, "thousand": 420, "twothousand": 1420, "fivethous": 3420, "tenthous": 3420, "odd100": 40, "even100": 41, "stringu1": "OBAAAA", "stringu2": "CAAAAA", "string4": "OOOOxx" }
+, { "unique1": 7164, "unique2": 4, "two": 0, "four": 0, "ten": 4, "twenty": 4, "hundred": 64, "thousand": 164, "twothousand": 1164, "fivethous": 2164, "tenthous": 7164, "odd100": 128, "even100": 129, "stringu1": "OPAAAA", "stringu2": "EAAAAA", "string4": "AAAAxx" }
+, { "unique1": 8800, "unique2": 0, "two": 0, "four": 0, "ten": 0, "twenty": 0, "hundred": 0, "thousand": 800, "twothousand": 800, "fivethous": 3800, "tenthous": 8800, "odd100": 0, "even100": 1, "stringu1": "MAAAAA", "stringu2": "AAAAAA", "string4": "AAAAxx" }
+, { "unique1": 4321, "unique2": 8, "two": 1, "four": 1, "ten": 1, "twenty": 1, "hundred": 21, "thousand": 321, "twothousand": 321, "fivethous": 4321, "tenthous": 4321, "odd100": 42, "even100": 43, "stringu1": "FKAAAA", "stringu2": "IAAAAA", "string4": "AAAAxx" }
+, { "unique1": 3043, "unique2": 9, "two": 1, "four": 3, "ten": 3, "twenty": 3, "hundred": 43, "thousand": 43, "twothousand": 1043, "fivethous": 3043, "tenthous": 3043, "odd100": 86, "even100": 87, "stringu1": "BNAAAA", "stringu2": "JAAAAA", "string4": "HHHHxx" }
+, { "unique1": 9850, "unique2": 3, "two": 0, "four": 2, "ten": 0, "twenty": 10, "hundred": 50, "thousand": 850, "twothousand": 1850, "fivethous": 4850, "tenthous": 9850, "odd100": 100, "even100": 101, "stringu1": "WOAAAA", "stringu2": "DAAAAA", "string4": "VVVVxx" }
+, { "unique1": 8009, "unique2": 5, "two": 1, "four": 1, "ten": 9, "twenty": 9, "hundred": 9, "thousand": 9, "twothousand": 9, "fivethous": 3009, "tenthous": 8009, "odd100": 18, "even100": 19, "stringu1": "BWAAAA", "stringu2": "FAAAAA", "string4": "HHHHxx" }
+, { "unique1": 6701, "unique2": 7, "two": 1, "four": 1, "ten": 1, "twenty": 1, "hundred": 1, "thousand": 701, "twothousand": 701, "fivethous": 1701, "tenthous": 6701, "odd100": 2, "even100": 3, "stringu1": "TXAAAA", "stringu2": "HAAAAA", "string4": "VVVVxx" }]);
+
+
+
+Upsert INTO tenk2 (
+    [{ "unique1": 5057, "unique2": 6, "two": 1, "four": 1, "ten": 7, "twenty": 17, "hundred": 57, "thousand": 57, "twothousand": 1057, "fivethous": 57, "tenthous": 5057, "odd100": 114, "even100": 115, "stringu1": "NMAAAA", "stringu2": "GAAAAA", "string4": "OOOOxx" }
+, { "unique1": 1891, "unique2": 1, "two": 1, "four": 3, "ten": 1, "twenty": 11, "hundred": 91, "thousand": 891, "twothousand": 1891, "fivethous": 1891, "tenthous": 1891, "odd100": 182, "even100": 183, "stringu1": "TUAAAA", "stringu2": "BAAAAA", "string4": "HHHHxx" }
+, { "unique1": 3420, "unique2": 2, "two": 0, "four": 0, "ten": 0, "twenty": 0, "hundred": 20, "thousand": 420, "twothousand": 1420, "fivethous": 3420, "tenthous": 3420, "odd100": 40, "even100": 41, "stringu1": "OBAAAA", "stringu2": "CAAAAA", "string4": "OOOOxx" }
+, { "unique1": 7164, "unique2": 4, "two": 0, "four": 0, "ten": 4, "twenty": 4, "hundred": 64, "thousand": 164, "twothousand": 1164, "fivethous": 2164, "tenthous": 7164, "odd100": 128, "even100": 129, "stringu1": "OPAAAA", "stringu2": "EAAAAA", "string4": "AAAAxx" }
+, { "unique1": 8800, "unique2": 0, "two": 0, "four": 0, "ten": 0, "twenty": 0, "hundred": 0, "thousand": 800, "twothousand": 800, "fivethous": 3800, "tenthous": 8800, "odd100": 0, "even100": 1, "stringu1": "MAAAAA", "stringu2": "AAAAAA", "string4": "AAAAxx" }
+, { "unique1": 4321, "unique2": 8, "two": 1, "four": 1, "ten": 1, "twenty": 1, "hundred": 21, "thousand": 321, "twothousand": 321, "fivethous": 4321, "tenthous": 4321, "odd100": 42, "even100": 43, "stringu1": "FKAAAA", "stringu2": "IAAAAA", "string4": "AAAAxx" }
+, { "unique1": 3043, "unique2": 9, "two": 1, "four": 3, "ten": 3, "twenty": 3, "hundred": 43, "thousand": 43, "twothousand": 1043, "fivethous": 3043, "tenthous": 3043, "odd100": 86, "even100": 87, "stringu1": "BNAAAA", "stringu2": "JAAAAA", "string4": "HHHHxx" }
+, { "unique1": 9850, "unique2": 3, "two": 0, "four": 2, "ten": 0, "twenty": 10, "hundred": 50, "thousand": 850, "twothousand": 1850, "fivethous": 4850, "tenthous": 9850, "odd100": 100, "even100": 101, "stringu1": "WOAAAA", "stringu2": "DAAAAA", "string4": "VVVVxx" }
+, { "unique1": 8009, "unique2": 5, "two": 1, "four": 1, "ten": 9, "twenty": 9, "hundred": 9, "thousand": 9, "twothousand": 9, "fivethous": 3009, "tenthous": 8009, "odd100": 18, "even100": 19, "stringu1": "BWAAAA", "stringu2": "FAAAAA", "string4": "HHHHxx" }
+, { "unique1": 6701, "unique2": 7, "two": 1, "four": 1, "ten": 1, "twenty": 1, "hundred": 1, "thousand": 701, "twothousand": 701, "fivethous": 1701, "tenthous": 6701, "odd100": 2, "even100": 3, "stringu1": "TXAAAA", "stringu2": "HAAAAA", "string4": "VVVVxx" }]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/secondary-heterogeneous-indexnl-params/secondary-heterogeneous-indexnl-params.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/secondary-heterogeneous-indexnl-params/secondary-heterogeneous-indexnl-params.3.query.sqlpp
new file mode 100644
index 0000000..c07004d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/secondary-heterogeneous-indexnl-params/secondary-heterogeneous-indexnl-params.3.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+/*
+ * indexnl hint. use specified index (idx_tenk2_1k_2k)
+ */
+
+select value tenk2.unique1
+from tenk1, tenk2
+where
+  tenk1.unique2 < 1
+and
+  tenk1.thousand /* +indexnl(idx_tenk2_1k_2k) */ = tenk2.thousand
+order by tenk2.unique1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/secondary-heterogeneous-indexnl-params/secondary-heterogeneous-indexnl-params.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/secondary-heterogeneous-indexnl-params/secondary-heterogeneous-indexnl-params.4.query.sqlpp
new file mode 100644
index 0000000..b5ac9ad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/secondary-heterogeneous-indexnl-params/secondary-heterogeneous-indexnl-params.4.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select value tenk2.unique1
+from tenk1, tenk2
+where
+  tenk1.unique2 < 2
+and
+  tenk1.thousand /* +indexnl() */ = tenk2.thousand
+order by tenk2.unique1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-Leftouterjoin/heterogeneous-index-Leftouterjoin.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-Leftouterjoin/heterogeneous-index-Leftouterjoin.1.ddl.sqlpp
new file mode 100644
index 0000000..200dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-Leftouterjoin/heterogeneous-index-Leftouterjoin.1.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE untyped AS {id: string};
+CREATE TYPE typed AS {id: string};
+
+CREATE DATASET ds_outer_untyped(untyped) primary key id;
+CREATE DATASET ds_outer_typed(typed) primary key id;
+
+CREATE DATASET ds_inner_untyped(untyped) primary key id;
+CREATE DATASET ds_inner_typed(typed) primary key id;
+
+CREATE INDEX idx_c_int32 ON ds_inner_untyped(c_int32);
+CREATE INDEX idx_c_int32 ON ds_inner_typed(c_int32);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-Leftouterjoin/heterogeneous-index-Leftouterjoin.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-Leftouterjoin/heterogeneous-index-Leftouterjoin.2.update.sqlpp
new file mode 100644
index 0000000..5b41d9b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-Leftouterjoin/heterogeneous-index-Leftouterjoin.2.update.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+USE test;
+
+UPSERT INTO ds_outer_untyped [
+{'id': "o_untyped:01", 'c_any':null, 'c_int8':null, 'c_int16':null, 'c_int32':null, 'c_int64':null, 'c_float':null, 'c_double':null}];
+
+UPSERT INTO ds_outer_typed [
+{'id': "o_untyped:01", 'c_any':null, 'c_int8':null, 'c_int16':null, 'c_int32':null, 'c_int64':null, 'c_float':null, 'c_double':null}];
+
+UPSERT INTO ds_inner_untyped [
+{ 'id': "i_untyped:01", 'c_any':null, 'c_int8':null, 'c_int16':null, 'c_int32':null, 'c_int64':null, 'c_float':null, 'c_double':null},
+{ 'id': "i_untyped:02", 'c_any':null, 'c_int8':null, 'c_int16':null, 'c_int32':null, 'c_int64':null, 'c_float':null, 'c_double':null},
+{ 'id': "i_untyped:03", 'c_any':null, 'c_int8':null, 'c_int16':null, 'c_int32':null, 'c_int64':null, 'c_float':null, 'c_double':null}
+];
+
+UPSERT INTO ds_inner_typed [
+{ 'id': "i_typed:01", 'c_any':null, 'c_int8':null, 'c_int16':null, 'c_int32':null, 'c_int64':null, 'c_float':null, 'c_double':null},
+{ 'id': "i_typed:02", 'c_any':null, 'c_int8':null, 'c_int16':null, 'c_int32':null, 'c_int64':null, 'c_float':null, 'c_double':null},
+{ 'id': "i_typed:03", 'c_any':null, 'c_int8':null, 'c_int16':null, 'c_int32':null, 'c_int64':null, 'c_float':null, 'c_double':null}
+];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-Leftouterjoin/heterogeneous-index-Leftouterjoin.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-Leftouterjoin/heterogeneous-index-Leftouterjoin.3.query.sqlpp
new file mode 100644
index 0000000..25d7c9d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-Leftouterjoin/heterogeneous-index-Leftouterjoin.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+SET `compiler.sort.parallel` 'false';
+SET `compiler.indexonly` 'true';
+SELECT t1.id AS t1_id, t2.id AS t2_id
+FROM ds_outer_untyped t1 LEFT JOIN ds_inner_typed t2 ON int32(t1.c_int32) /* +indexnl */ = (t2.c_int32) ;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-Leftouterjoin/heterogeneous-index-Leftouterjoin.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-Leftouterjoin/heterogeneous-index-Leftouterjoin.4.query.sqlpp
new file mode 100644
index 0000000..065273c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-Leftouterjoin/heterogeneous-index-Leftouterjoin.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+SET `compiler.sort.parallel` 'false';
+SET `compiler.indexonly` 'false';
+SELECT t1.id AS t1_id, t2.id AS t2_id
+FROM ds_outer_untyped t1 LEFT JOIN ds_inner_typed t2 ON int32(t1.c_int32) /* +indexnl */ = (t2.c_int32);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-bulkLoad/heterogeneous-index-bulkLoad.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-bulkLoad/heterogeneous-index-bulkLoad.1.ddl.sqlpp
new file mode 100644
index 0000000..d425059
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-bulkLoad/heterogeneous-index-bulkLoad.1.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+
+drop  dataverse test if exists;
+create  dataverse test;
+use test;
+
+CREATE TYPE UserType AS {
+  userId : int,
+  email : string,
+  username:string,
+  isActive : boolean };
+
+create dataset UserTypes(UserType) primary key userId;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-bulkLoad/heterogeneous-index-bulkLoad.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-bulkLoad/heterogeneous-index-bulkLoad.2.update.sqlpp
new file mode 100644
index 0000000..05a21fa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-bulkLoad/heterogeneous-index-bulkLoad.2.update.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use test;
+
+load  dataset UserTypes using localfs ((`path`=`asterix_nc1://data/heterogeneousData.adm`),(`format`=`adm`));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-bulkLoad/heterogeneous-index-bulkLoad.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-bulkLoad/heterogeneous-index-bulkLoad.3.ddl.sqlpp
new file mode 100644
index 0000000..027d408
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-bulkLoad/heterogeneous-index-bulkLoad.3.ddl.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+create index idx_user_age  on UserTypes(age);
+create index idx_user_age_occupation1  on UserTypes(age, occupation);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-bulkLoad/heterogeneous-index-bulkLoad.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-bulkLoad/heterogeneous-index-bulkLoad.4.query.sqlpp
new file mode 100644
index 0000000..944054a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-bulkLoad/heterogeneous-index-bulkLoad.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+select *
+from UserTypes u
+where u.age > 20
+order by userId;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-select/heterogeneous-index-select.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-select/heterogeneous-index-select.1.ddl.sqlpp
new file mode 100644
index 0000000..e0e6961
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-select/heterogeneous-index-select.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+drop  dataverse test if exists;
+create  dataverse test;
+use test;
+
+CREATE TYPE UserType AS {
+  userId : int,
+  email : string,
+  username:string,
+  isActive : boolean };
+
+create dataset UserTypes(UserType) primary key userId;
+create dataset UserTypes2(UserType) primary key userId;
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-select/heterogeneous-index-select.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-select/heterogeneous-index-select.2.ddl.sqlpp
new file mode 100644
index 0000000..2d84c34
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-select/heterogeneous-index-select.2.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+use test;
+
+create index idx_user_age  on UserTypes(age);
+create index idx_user_age_occupation1  on UserTypes(age, occupation);
+create index idx_user_age_occupation2  on UserTypes2(age, occupation);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-select/heterogeneous-index-select.3.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-select/heterogeneous-index-select.3.update.sqlpp
new file mode 100644
index 0000000..17fb100
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-select/heterogeneous-index-select.3.update.sqlpp
@@ -0,0 +1,221 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+Use test;
+
+
+INSERT INTO UserTypes (
+    [
+        {
+            "userId": 1,
+            "email": "john.doe@example.com",
+            "username": "johndoe",
+            "isActive": true,
+            "occupation": "teacher",
+            "age": 23
+        },
+        {
+            "userId": 2,
+            "email": "jane.smith@example.com",
+            "username": "janesmith",
+            "isActive": false,
+            "occupation": "firefighter"
+
+
+        },
+        {
+            "userId": 3,
+            "email": "michael.brown@example.com",
+            "username": "michaelb",
+            "isActive": true,
+            "age": "five",
+            "occupation":4
+
+        },
+        {
+            "userId": 4,
+            "email": "emily.davis@example.com",
+            "username": "emilyd",
+            "isActive": true,
+            "age": 23.1
+
+        },
+        {
+            "userId": 5,
+            "email": "chris.jones@example.com",
+            "username": "chrisj",
+            "isActive": false,
+            "age": "old"
+
+        },
+        {
+            "userId": 6,
+            "email": "patricia.garcia@example.com",
+            "username": "patriciag",
+            "isActive": true,
+            "age": "45"
+
+        },
+        {
+            "userId": 7,
+            "email": "linda.martinez@example.com",
+            "username": "lindam",
+            "isActive": false
+        },
+        {
+            "userId": 8,
+            "email": "robert.lee@example.com",
+            "username": "robertl",
+            "isActive": true,
+            "age": 31
+
+        },
+        {
+            "userId": 9,
+            "email": "james.wilson@example.com",
+            "username": "jamesw",
+            "isActive": true,
+            "age": "65"
+
+        },
+        {
+            "userId": 10,
+            "email": "mary.moore@example.com",
+            "username": "marym",
+            "isActive": false,
+            "age": ""
+
+        },
+        {
+            "userId": 11,
+            "email": "jack.moore@example.com",
+            "username": "jk",
+            "isActive": true,
+            "age": "20",
+            "occupation": "firefighter"
+
+
+        }
+    ]
+);
+
+
+
+
+INSERT INTO UserTypes2 (
+    [
+        {
+            "userId": 1,
+            "email": "john.doe@example.com",
+            "username": "johndoe",
+            "isActive": true,
+            "occupation": "teacher",
+            "age": 23
+        },
+        {
+            "userId": 2,
+            "email": "jane.smith@example.com",
+            "username": "janesmith",
+            "isActive": false,
+            "occupation": "firefighter"
+
+
+        },
+        {
+            "userId": 3,
+            "email": "michael.brown@example.com",
+            "username": "michaelb",
+            "isActive": true,
+            "age": "five",
+            "occupation":4
+
+        },
+        {
+            "userId": 4,
+            "email": "emily.davis@example.com",
+            "username": "emilyd",
+            "isActive": true,
+            "age": 23.1
+
+        },
+        {
+            "userId": 5,
+            "email": "chris.jones@example.com",
+            "username": "chrisj",
+            "isActive": false,
+            "age": "old"
+
+        },
+        {
+            "userId": 6,
+            "email": "patricia.garcia@example.com",
+            "username": "patriciag",
+            "isActive": true,
+            "age": "45"
+
+        },
+        {
+            "userId": 7,
+            "email": "linda.martinez@example.com",
+            "username": "lindam",
+            "isActive": false
+        },
+        {
+            "userId": 8,
+            "email": "robert.lee@example.com",
+            "username": "robertl",
+            "isActive": true,
+            "age": 31
+
+        },
+        {
+            "userId": 9,
+            "email": "james.wilson@example.com",
+            "username": "jamesw",
+            "isActive": true,
+            "age": "65"
+
+        },
+        {
+            "userId": 10,
+            "email": "mary.moore@example.com",
+            "username": "marym",
+            "isActive": false,
+            "age": ""
+
+        },
+        {
+            "userId": 11,
+            "email": "jack.moore@example.com",
+            "username": "jk",
+            "isActive": true,
+            "age": "20",
+            "occupation": "firefighter"
+
+
+        }
+    ]
+);
+
+
+
+
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-select/heterogeneous-index-select.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-select/heterogeneous-index-select.4.query.sqlpp
new file mode 100644
index 0000000..40e8c99
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-select/heterogeneous-index-select.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+Use test;
+
+select *
+from UserTypes u
+where u.age > 20
+order by userId;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-select/heterogeneous-index-select.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-select/heterogeneous-index-select.5.query.sqlpp
new file mode 100644
index 0000000..cf04d96
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-select/heterogeneous-index-select.5.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+Use test;
+
+
+
+select *
+from UserTypes u
+where u.age > 20 and u.isActive = true
+order by userId;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-select/heterogeneous-index-select.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-select/heterogeneous-index-select.6.query.sqlpp
new file mode 100644
index 0000000..4d6d0a2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-select/heterogeneous-index-select.6.query.sqlpp
@@ -0,0 +1,26 @@
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+Use test;
+
+select *
+from UserTypes u
+where u.age = "20" and u.occupation = "firefighter"
+order by userId;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-select/heterogeneous-index-select.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-select/heterogeneous-index-select.7.query.sqlpp
new file mode 100644
index 0000000..45fa384
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-select/heterogeneous-index-select.7.query.sqlpp
@@ -0,0 +1,26 @@
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+Use test;
+
+select *
+from UserTypes2 u
+where u.age = "20" and u.occupation = "firefighter"
+order by userId;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-type-and-heterogeneous-key/heterogeneous-index-type-and-heterogeneous-key.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-type-and-heterogeneous-key/heterogeneous-index-type-and-heterogeneous-key.1.ddl.sqlpp
new file mode 100644
index 0000000..9c4994a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-type-and-heterogeneous-key/heterogeneous-index-type-and-heterogeneous-key.1.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+drop  dataverse test if exists;
+create  dataverse test;
+use test;
+
+
+CREATE TYPE t1 AS {
+  id: int,
+  f2: int};
+
+CREATE DATASET ds(t1) primary key id;
+CREATE INDEX i2 on ds(f1,f2);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-type-and-heterogeneous-key/heterogeneous-index-type-and-heterogeneous-key.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-type-and-heterogeneous-key/heterogeneous-index-type-and-heterogeneous-key.2.ddl.sqlpp
new file mode 100644
index 0000000..5f364b1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/heterogeneous-index-type-and-heterogeneous-key/heterogeneous-index-type-and-heterogeneous-key.2.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+drop  dataverse test if exists;
+create  dataverse test;
+use test;
+
+
+CREATE TYPE t1 AS {
+  id: int,
+  f2: int};
+
+CREATE DATASET ds(t1) primary key id;
+CREATE INDEX i2 on ds(f1,f3: int);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.01.ddl.sqlpp
new file mode 100644
index 0000000..8a1a1d9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.01.ddl.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE t1 AS { id: int,name: string, age: int };
+
+CREATE DATASET ds1(t1) PRIMARY KEY id;
+
+CREATE INDEX ds1_name ON ds1(name: string);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.02.update.sqlpp
new file mode 100644
index 0000000..8f4a38b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.02.update.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+UPSERT INTO ds1 ([
+{"id":1, "name": "John", "age": 21},
+{"id":2, "name": "asd gdh", "age": 23},
+{"id":3, "name": "xossted", "age": 102},
+{"id":4, "name": "asdgdh👩‍👩‍👧dfsd", "age": 23},
+{"id":5, "name": "sxvciis", "age": 13},
+{"id":6, "name": "fsdxv😀", "age": 17},
+{"id":7, "name": "fsdxv😁", "age": 17},
+{"id":8, "name": "Join", "age": 21},
+{"id":9, "name": "sxvfjshfjisciis", "age": 13},
+{"id":10, "name": "h\\axx%", "age": 123},
+{"id":11, "name": "h\\\\%axx%", "age": 11},
+{"id":12, "name": "h\\\\_axx%", "age": 43},
+{"id":13, "name": "h\\\\ax\\%", "age": 21},
+{"id":14, "name": "h\\\\ax\\%", "age": 4},
+{"id":15, "name": "ax\\_%", "age": 6},
+{"id":16, "name": "axnvtxb", "age": 53},
+{"id":17, "name": "fvbn\"", "age": 23},
+{"id":18, "name": "fv%%\\bn\"", "age": 99},
+{"id":19, "name": "fv%%\\bn\"", "age": 101}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.03.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.03.query.sqlpp
new file mode 100644
index 0000000..c93433e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.03.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select * from ds1 where name like "fsdxv😀%";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.04.query.sqlpp
new file mode 100644
index 0000000..94b9911
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.04.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+explain select * from ds1 where name like "fsdxv😀%";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.05.query.sqlpp
new file mode 100644
index 0000000..05f4fcb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.05.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select * from ds1 where name like "asdgdh👩‍👩‍👧%";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.06.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.06.query.sqlpp
new file mode 100644
index 0000000..712f952
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.06.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+explain select * from ds1 where name like "asdgdh👩‍👩‍👧%";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.07.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.07.query.sqlpp
new file mode 100644
index 0000000..e196c9c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.07.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select * from ds1 where name like "sxvciis";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.08.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.08.query.sqlpp
new file mode 100644
index 0000000..8727844
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.08.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+explain select * from ds1 where name like "sxvciis";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.09.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.09.query.sqlpp
new file mode 100644
index 0000000..9e40b15
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.09.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select * from ds1 where name like "Joh%";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.10.query.sqlpp
new file mode 100644
index 0000000..c5272bb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.10.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+explain select * from ds1 where name like "Joh%";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.11.query.sqlpp
new file mode 100644
index 0000000..f4af3ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.11.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select * from ds1 where name like "s%vfjs__jiscii%";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.12.query.sqlpp
new file mode 100644
index 0000000..712538d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.12.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+explain select * from ds1 where name like "s%vfjs__jiscii%";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.13.query.sqlpp
new file mode 100644
index 0000000..a4a194c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.13.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select * from ds1 where name like "sxvc_is";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.14.query.sqlpp
new file mode 100644
index 0000000..f191e96
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.14.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+explain select * from ds1 where name like "sxvc_is";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.15.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.15.query.sqlpp
new file mode 100644
index 0000000..bc095fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.15.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select * from ds1 where name like "sxvc_i%";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.16.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.16.query.sqlpp
new file mode 100644
index 0000000..15de270
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.16.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+explain select * from ds1 where name like "sxvc_i%";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.17.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.17.query.sqlpp
new file mode 100644
index 0000000..c7370c5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.17.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select * from ds1 where name like "sxvfjs%jisci%";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.18.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.18.query.sqlpp
new file mode 100644
index 0000000..915e4ac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.18.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+explain select * from ds1 where name like "sxvfjs%jisci%";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.19.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.19.query.sqlpp
new file mode 100644
index 0000000..81ea8e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.19.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select * from ds1 where name like "J%";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.20.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.20.query.sqlpp
new file mode 100644
index 0000000..c6f30c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.20.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+explain select * from ds1 where name like "J%";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.21.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.21.query.sqlpp
new file mode 100644
index 0000000..4cddaff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.21.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select * from ds1 where name like "sx%";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.22.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.22.query.sqlpp
new file mode 100644
index 0000000..4a02f4d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.22.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+explain select * from ds1 where name like "sx%";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.23.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.23.query.sqlpp
new file mode 100644
index 0000000..cbeca01
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.23.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+{
+"t1": (select * from [{"x":"fvxb%"}] p where x like "fvxb\\%"),
+"t2": (select * from [{"x":"fv%%\\bn\\%"}] p where x like "fv\\%\\%\\\\bn\\\\%"),
+"t3": (select * from [{"x":"fvxbx"}] p where x like "fvxb\\%"),
+"t4": (select * from [{"x":"fv%xbx"}] p where x like "fv\\%xb%"),
+"t5": (select * from [{"x":"a\\%bcd"}] p where x like "a\\\\\\%bcd"),
+"t6": (select * from [{"x":"abc\n%"}] p where x like "abc\n\\%"),
+"t7": (select * from [{"x":"abc\n%"}] p where x like "abc\n%"),
+"t8": (select * from [{"x":"abc%cbz"},{"x":"abc%%x"},{"x":"abc"}] p where x like "abc\\%%"),
+"t9": (select * from [{"x":"vhjdbvsabc_fjdhf"},{"x":"abc%%x"},{"x":"abc"}] p where x like "%abc\\_%"),
+"t10": (select * from [{"x":"abc_fjdhf"},{"x":"abc%%x"},{"x":"abc"}] p where x like "abc\\_%"),
+"t11": (select * from [{"x":"abc_fjdhf"},{"x":"abc%x"},{"x":"abc%yz"}] p where x like "abc\\%_"),
+"t12": (select * from [{"x":"abc_fjdhf"},{"x":"abc%x"},{"x":"abc%yz"},{"x":"abc_"}] p where x like "abc\\_")
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.24.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.24.query.sqlpp
new file mode 100644
index 0000000..3657c3b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.24.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select * from ds1 where name like "h\\\\axx\\%";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.25.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.25.query.sqlpp
new file mode 100644
index 0000000..0cb91aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/like-expression/like-expression.25.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+explain select * from ds1 where name like "h\\\\axx\\%";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3410/query-ASTERIXDB-3410.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3410/query-ASTERIXDB-3410.1.ddl.sqlpp
new file mode 100644
index 0000000..4271f99
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3410/query-ASTERIXDB-3410.1.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: This test case is to verify the fix for ASTERIXDB-3419
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type dt1 as {itemid: int};
+
+create dataset collection1(dt1) PRIMARY KEY itemid;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3410/query-ASTERIXDB-3410.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3410/query-ASTERIXDB-3410.2.update.sqlpp
new file mode 100644
index 0000000..d805bd0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3410/query-ASTERIXDB-3410.2.update.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into collection1
+([
+    {
+       "categories": "Category 1, Category 2, Category 6, Category 9, Category 10, Category 12, Category 13, Category 14, Category 16, Category 17, Category 18, Category 19, Category 20, Category 21, Category 22",
+       "itemid":10,
+       "description":"ABC"
+    },
+    {
+           "categories": "Category 1, Category 3, Category 5, Category 7, Category 8, Category 9, Category 10, Category 19, Category 10, Category 20, Category 21, Category 22, Category 23, Category 31, Category 32",
+           "itemid":12,
+           "description":"XYZ"
+        }
+]);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3410/query-ASTERIXDB-3410.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3410/query-ASTERIXDB-3410.3.query.sqlpp
new file mode 100644
index 0000000..9081a11
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3410/query-ASTERIXDB-3410.3.query.sqlpp
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: This test case is to verify the fix for ASTERIXDB-3410
+ */
+
+use test;
+
+SELECT VALUE OBJECT_REMOVE(t, 'categories')
+FROM (
+  SELECT t.*,
+         to_bigint(CONTAINS(categories, "Category 1")) AS `Category 1`
+  FROM (
+    SELECT t.*,
+           to_bigint(CONTAINS(categories, "Category 2")) AS `Category 2`
+    FROM (
+      SELECT t.*,
+             to_bigint(CONTAINS(categories, "Category 3")) AS `Category 3`
+      FROM (
+        SELECT t.*,
+               to_bigint(CONTAINS(categories, "Category 4")) AS `Category 4`
+        FROM (
+          SELECT t.*,
+                 to_bigint(CONTAINS(categories, "Category 5")) AS `Category 5`
+          FROM (
+            SELECT t.*,
+                   to_bigint(CONTAINS(categories, "Category 6")) AS `Category 6`
+            FROM (
+              SELECT t.*,
+                     to_bigint(CONTAINS(categories, "Category 7")) AS `Category 7`
+              FROM (
+                SELECT t.*,
+                       to_bigint(CONTAINS(categories, "Category 8")) AS `Category 8`
+                FROM (
+                  SELECT t.*,
+                         to_bigint(CONTAINS(categories, "Category 9")) AS `Category 9`
+                  FROM (
+                    SELECT t.*,
+                           to_bigint(CONTAINS(categories, "Category 10")) AS `Category 10`
+                    FROM (
+                      SELECT t.*,
+                             to_bigint(CONTAINS(categories, "Category 11")) AS `Category 11`
+                      FROM (
+                        SELECT t.*,
+                               to_bigint(CONTAINS(categories, "Category 12")) AS `Category 12`
+                        FROM (
+                          SELECT t.*,
+                                 to_bigint(CONTAINS(categories, "Category 13")) AS `Category 13`
+                          FROM (
+                            SELECT t.*,
+                                   to_bigint(CONTAINS(categories, "Category 14")) AS `Category 14`
+                            FROM (
+                              SELECT t.*,
+                                     to_bigint(CONTAINS(categories, "Category 15")) AS `Category 15`
+                              FROM (
+                                SELECT t.*,
+                                       to_bigint(CONTAINS(categories, "Category 16")) AS `Category 16`
+                                FROM (
+                                  SELECT t.*,
+                                         to_bigint(CONTAINS(categories, "Category 17")) AS `Category 17`
+                                  FROM (
+                                    SELECT t.*,
+                                           to_bigint(CONTAINS(categories, "Category 18")) AS `Category 18`
+                                    FROM (
+                                      SELECT t.*,
+                                             to_bigint(CONTAINS(categories, "Category 19")) AS `Category 19`
+                                      FROM (
+                                        SELECT t.*,
+                                               to_bigint(CONTAINS(categories, "Category 20")) AS `Category 20`
+                                        FROM (
+                                          SELECT t.*,
+                                                 to_bigint(CONTAINS(categories, "Category 21")) AS `Category 21`
+                                          FROM (
+                                            SELECT t.*,
+                                                   to_bigint(CONTAINS(categories, "Category 22")) AS `Category 22`
+                                            FROM (
+                                              SELECT t.*,
+                                                     to_bigint(CONTAINS(categories, "Category 23")) AS `Category 23`
+                                              FROM (
+                                                SELECT t.*,
+                                                       to_bigint(CONTAINS(categories, "Category 24")) AS `Category 24`
+                                                FROM (
+                                                  SELECT t.*,
+                                                         to_bigint(CONTAINS(categories, "Category 25")) AS `Category 25`
+                                                  FROM (
+                                                    SELECT t.*,
+                                                           to_bigint(CONTAINS(categories, "Category 26")) AS `Category 26`
+                                                    FROM collection1 t ) AS t ) AS t ) AS t ) AS t ) AS t ) AS t ) AS t ) AS t ) AS t ) AS t ) AS t ) AS t ) AS t ) AS t ) AS t ) AS t ) AS t ) AS t ) AS t ) AS t ) AS t ) AS t ) AS t ) AS t ) AS t ) AS t ORDER BY itemid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/var_star/var_star.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/var_star/var_star.5.query.sqlpp
new file mode 100644
index 0000000..7a94768
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/var_star/var_star.5.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE tpch;
+
+SELECT a.b.* from [{"b":1}, {"b":{"c":2}}, null] a
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/var_star/var_star.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/var_star/var_star.6.query.sqlpp
new file mode 100644
index 0000000..da33d6d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/var_star/var_star.6.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE tpch;
+
+SELECT a.b.*, d.* from [{"b":1}, {"b":{"c":2}}, null] a, [{"b":1}] d;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-does-not-exist/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-does-not-exist/test.040.query.sqlpp
new file mode 100644
index 0000000..dc66c29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-does-not-exist/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "doesNotExistCollection") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-does-not-exist/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-does-not-exist/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-does-not-exist/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-exists/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-exists/test.010.ddl.sqlpp
new file mode 100644
index 0000000..7544a66
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-exists/test.010.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE testScope IF EXISTS;
+CREATE DATAVERSE testScope;
+USE testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-exists/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-exists/test.020.update.sqlpp
new file mode 100644
index 0000000..711dd3f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-exists/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-exists/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-exists/test.040.query.sqlpp
new file mode 100644
index 0000000..d4442c7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-exists/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "testCollection") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-exists/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-exists/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/collection-exists/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/external-collection/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/external-collection/test.010.ddl.sqlpp
new file mode 100644
index 0000000..5dcc7e0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/external-collection/test.010.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE testScope IF EXISTS;
+CREATE DATAVERSE testScope;
+USE testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
+
+CREATE TYPE test AS open {};
+CREATE EXTERNAL DATASET testExternalCollection(test) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("sessionToken"="dummySessionToken"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://127.0.0.1:8001"),
+("container"="playground"),
+("definition"="json-data/reviews/single-line/json"),
+("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/external-collection/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/external-collection/test.020.update.sqlpp
new file mode 100644
index 0000000..711dd3f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/external-collection/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/external-collection/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/external-collection/test.040.query.sqlpp
new file mode 100644
index 0000000..9915cfd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/external-collection/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "testExternalCollection") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/external-collection/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/external-collection/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/external-collection/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.010.ddl.sqlpp
new file mode 100644
index 0000000..7544a66
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.010.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE testScope IF EXISTS;
+CREATE DATAVERSE testScope;
+USE testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.020.update.sqlpp
new file mode 100644
index 0000000..711dd3f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.021.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.021.update.sqlpp
new file mode 100644
index 0000000..24916e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.021.update.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testScope;
+
+CREATE INDEX testCollection_idx ON testCollection(some_value: int);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.040.query.sqlpp
new file mode 100644
index 0000000..905a20a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "testCollection", "testCollection_idx_fake") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-does-not-exist/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.010.ddl.sqlpp
new file mode 100644
index 0000000..7544a66
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.010.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE testScope IF EXISTS;
+CREATE DATAVERSE testScope;
+USE testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.020.update.sqlpp
new file mode 100644
index 0000000..711dd3f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.021.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.021.update.sqlpp
new file mode 100644
index 0000000..24916e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.021.update.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testScope;
+
+CREATE INDEX testCollection_idx ON testCollection(some_value: int);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.040.query.sqlpp
new file mode 100644
index 0000000..cb60cb9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "testCollection", "testCollection_idx") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/index-exists/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.010.ddl.sqlpp
new file mode 100644
index 0000000..7544a66
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.010.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE testScope IF EXISTS;
+CREATE DATAVERSE testScope;
+USE testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.020.update.sqlpp
new file mode 100644
index 0000000..711dd3f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.040.query.sqlpp
new file mode 100644
index 0000000..d376541
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.041.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.041.query.sqlpp
new file mode 100644
index 0000000..b395ed5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.041.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "testCollection", "testIndex", "tooManyArgs") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/invalid-arguments-count/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-constant-argument/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-constant-argument/test.010.ddl.sqlpp
new file mode 100644
index 0000000..7544a66
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-constant-argument/test.010.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE testScope IF EXISTS;
+CREATE DATAVERSE testScope;
+USE testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-constant-argument/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-constant-argument/test.020.update.sqlpp
new file mode 100644
index 0000000..711dd3f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-constant-argument/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-constant-argument/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-constant-argument/test.040.query.sqlpp
new file mode 100644
index 0000000..2a0260e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-constant-argument/test.040.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testScope;
+SELECT storage_size(v.id, v.id, v.id) FROM testCollection AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-constant-argument/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-constant-argument/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-constant-argument/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-string-argument/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-string-argument/test.010.ddl.sqlpp
new file mode 100644
index 0000000..7544a66
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-string-argument/test.010.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE testScope IF EXISTS;
+CREATE DATAVERSE testScope;
+USE testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-string-argument/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-string-argument/test.020.update.sqlpp
new file mode 100644
index 0000000..711dd3f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-string-argument/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-string-argument/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-string-argument/test.040.query.sqlpp
new file mode 100644
index 0000000..ddb1996
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-string-argument/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT VALUE v FROM storage_size(1, 2, 3) AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-string-argument/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-string-argument/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/non-string-argument/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/sanity/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/sanity/test.010.ddl.sqlpp
new file mode 100644
index 0000000..761ae9f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/sanity/test.010.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE testScope IF EXISTS;
+CREATE DATAVERSE testScope;
+USE testScope;
+
+CREATE COLLECTION testCollection PRIMARY KEY (id: int);
+
+CREATE INDEX testCollection_idx ON testCollection(someField1: int);
+
+CREATE COLLECTION collectionAndIndexes PRIMARY KEY (id: int);
+CREATE COLLECTION collectionOnly PRIMARY KEY (id: int);
+CREATE COLLECTION indexOnly PRIMARY KEY (id: int);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/sanity/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/sanity/test.020.update.sqlpp
new file mode 100644
index 0000000..44905ff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/sanity/test.020.update.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {
+    "id": intVal,
+    "someField1": intVal,
+    "someField2": uuid(),
+    "someField3": {"firstName": uuid(), "lastName": uuid(), "location": {"lon": intVal * 2 + intVal, "lat": intVal * 7 + intVal}}
+    }
+    FROM range(1, 100000) intVal
+);
+
+INSERT INTO collectionAndIndexes({"id": 1, "valueField": (select value size from storage_size("testDatabase", "testScope", "testCollection") as x)[0]});
+INSERT INTO collectionOnly({"id": 1, "valueField": (select value size from storage_size("testDatabase", "testScope", "testCollection", "testCollection") as x)[0]});
+INSERT INTO indexOnly({"id": 1, "valueField": (select value size from storage_size("testDatabase", "testScope", "testCollection", "testCollection_idx") as x)[0]});
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/sanity/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/sanity/test.040.query.sqlpp
new file mode 100644
index 0000000..4ad1a6e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/sanity/test.040.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testScope;
+
+SELECT a.valueField > b.valueField AS collectionAndIndexesLargerThanCollectionOnly, b.valueField > c.valueField AS collectionOnlyLargerThanIndexOnly
+FROM collectionAndIndexes a,
+     collectionOnly b,
+     indexOnly c
+WHERE a.id = b.id
+  AND a.id = c.id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/sanity/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/sanity/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/sanity/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/view-not-collection/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/view-not-collection/test.010.ddl.sqlpp
new file mode 100644
index 0000000..34a3239
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/view-not-collection/test.010.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE testScope IF EXISTS;
+CREATE DATAVERSE testScope;
+USE testScope;
+
+CREATE COLLECTION testCollection
+PRIMARY KEY (id: int);
+
+CREATE VIEW testCollection_vw(id int)
+DEFAULT NULL datetime "YYYY-MM-DD hh:mm:ss" AS
+SELECT t.id
+FROM testCollection t;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/view-not-collection/test.020.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/view-not-collection/test.020.update.sqlpp
new file mode 100644
index 0000000..711dd3f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/view-not-collection/test.020.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE testScope;
+
+INSERT INTO testCollection(
+    SELECT VALUE {"id": intVal, "some_value": intVal}
+    FROM range(1, 512) intVal
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/view-not-collection/test.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/view-not-collection/test.040.query.sqlpp
new file mode 100644
index 0000000..cd88a1e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/view-not-collection/test.040.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT VALUE v FROM storage_size("testDatabase", "testScope", "testCollection_vw") AS v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/view-not-collection/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/view-not-collection/test.999.ddl.sqlpp
new file mode 100644
index 0000000..10b92b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/storage-size/datasource-function/view-not-collection/test.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATABASE testDatabase IF EXISTS;
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 f15e0d9..d0efadf 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
@@ -12,19 +12,15 @@
           -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$27]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select (and(ge($$25, 1), le($$25, 10))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  project ([$$27, $$25]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$25] <- [$$Tweet.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
+              select (and(ge($$25, 1), le($$25, 10))) project: [$$27] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                assign [$$25] <- [$$Tweet.getField(1)] project: [$$27, $$25] [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]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$27, $$Tweet] <- Twitter.Tweet [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|
-                        data-scan []<-[$$27, $$Tweet] <- Twitter.Tweet [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|
+                        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-sql/countn_01/countn_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/countn_01/countn_01.1.adm
new file mode 100644
index 0000000..00750ed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/countn_01/countn_01.1.adm
@@ -0,0 +1 @@
+3
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/countn_02/countn_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/countn_02/countn_02.1.adm
new file mode 100644
index 0000000..565b903
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/countn_02/countn_02.1.adm
@@ -0,0 +1,7 @@
+3
+2
+1
+1
+0
+0
+0
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/countn_distinct/countn_distinct.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/countn_distinct/countn_distinct.1.adm
new file mode 100644
index 0000000..c1f0a1c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/countn_distinct/countn_distinct.1.adm
@@ -0,0 +1 @@
+{ "t1": 3, "t2": 0, "t3": 2, "t4": 3, "t5": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/countn_empty_01/countn_empty_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/countn_empty_01/countn_empty_01.1.adm
new file mode 100644
index 0000000..c227083
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/countn_empty_01/countn_empty_01.1.adm
@@ -0,0 +1 @@
+0
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/countn_empty_02/countn_empty_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/countn_empty_02/countn_empty_02.1.adm
new file mode 100644
index 0000000..573541a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/countn_empty_02/countn_empty_02.1.adm
@@ -0,0 +1 @@
+0
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/countn_null/countn_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/countn_null/countn_null.1.adm
new file mode 100644
index 0000000..3c4c7ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/countn_null/countn_null.1.adm
@@ -0,0 +1 @@
+{ "sql-count": 3 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/scalar_countn/scalar_count.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/scalar_countn/scalar_count.1.adm
new file mode 100644
index 0000000..3230e04
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/scalar_countn/scalar_count.1.adm
@@ -0,0 +1,7 @@
+3
+3
+3
+3
+3
+3
+0
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/scalar_countn_empty/scalar_countn_empty.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/scalar_countn_empty/scalar_countn_empty.1.adm
new file mode 100644
index 0000000..573541a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/scalar_countn_empty/scalar_countn_empty.1.adm
@@ -0,0 +1 @@
+0
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/scalar_countn_null/scalar_countn_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/scalar_countn_null/scalar_countn_null.1.adm
new file mode 100644
index 0000000..b4f89e0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/scalar_countn_null/scalar_countn_null.1.adm
@@ -0,0 +1,10 @@
+3
+3
+3
+3
+3
+3
+0
+0
+0
+0
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 b3eb407..cc00b25 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
@@ -12,19 +12,15 @@
           -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$27]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select (and(ge($$25, 1), le($$25, 10))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  project ([$$27, $$25]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$25] <- [$$Tweet.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
+              select (and(ge($$25, 1), le($$25, 10))) project: [$$27] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                assign [$$25] <- [$$Tweet.getField(1)] project: [$$27, $$25] [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]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$27, $$Tweet] <- Twitter.Tweet [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|
-                        data-scan []<-[$$27, $$Tweet] <- Twitter.Tweet [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|
+                        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/countn_01/countn_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/countn_01/countn_01.1.adm
new file mode 100644
index 0000000..1d6b377
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/countn_01/countn_01.1.adm
@@ -0,0 +1,4 @@
+3
+3
+{ "$1": 2, "n": "a" }
+{ "$1": 1, "n": "b" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/countn_02/countn_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/countn_02/countn_02.1.adm
new file mode 100644
index 0000000..65ede1c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/countn_02/countn_02.1.adm
@@ -0,0 +1,7 @@
+3
+null
+1
+null
+null
+0
+0
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/countn_distinct/countn_distinct.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/countn_distinct/countn_distinct.1.adm
new file mode 100644
index 0000000..ae0b195
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/countn_distinct/countn_distinct.1.adm
@@ -0,0 +1 @@
+{ "t1": 3, "t2": null, "t3": null, "t4": 3, "t5": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/countn_empty_01/countn_empty_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/countn_empty_01/countn_empty_01.1.adm
new file mode 100644
index 0000000..573541a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/countn_empty_01/countn_empty_01.1.adm
@@ -0,0 +1 @@
+0
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/countn_empty_02/countn_empty_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/countn_empty_02/countn_empty_02.1.adm
new file mode 100644
index 0000000..c227083
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/countn_empty_02/countn_empty_02.1.adm
@@ -0,0 +1 @@
+0
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/countn_null/countn_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/countn_null/countn_null.1.adm
new file mode 100644
index 0000000..5fb8b09
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/countn_null/countn_null.1.adm
@@ -0,0 +1 @@
+{ "count": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/scalar_countn/scalar_countn.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/scalar_countn/scalar_countn.1.adm
new file mode 100644
index 0000000..3230e04
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/scalar_countn/scalar_countn.1.adm
@@ -0,0 +1,7 @@
+3
+3
+3
+3
+3
+3
+0
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/scalar_countn_empty/scalar_countn_empty.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/scalar_countn_empty/scalar_countn_empty.1.adm
new file mode 100644
index 0000000..c227083
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/scalar_countn_empty/scalar_countn_empty.1.adm
@@ -0,0 +1 @@
+0
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/scalar_countn_null/scalar_countn_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/scalar_countn_null/scalar_countn_null.1.adm
new file mode 100644
index 0000000..b533851
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/scalar_countn_null/scalar_countn_null.1.adm
@@ -0,0 +1,10 @@
+null
+null
+null
+null
+null
+null
+null
+null
+0
+0
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cache-residency/cache-residency.002.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cache-residency/cache-residency.002.regexjson
index b3f8c7c..d478e8a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cache-residency/cache-residency.002.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cache-residency/cache-residency.002.regexjson
@@ -16,6 +16,6 @@
 		"resultSize": "R{.*}",
 		"processedObjects": "R{.*}",
 		"bufferCacheHitRatio": "100.00%",
-		"bufferCachePageReadCount": 1
+		"bufferCachePageReadCount": "R{.*}"
 	}
 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
index 52981ae..7c963a7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
@@ -41,11 +41,13 @@
     "compiler\.indexonly" : true,
     "compiler\.internal\.sanitycheck" : true,
     "compiler\.joinmemory" : 262144,
+    "compiler\.max\.variable\.occurrences\.inlining" : 128,
     "compiler.min.groupmemory" : 524288,
     "compiler.min.joinmemory" : 524288,
     "compiler\.min\.memory\.allocation" : true,
     "compiler.min.sortmemory" : 524288,
     "compiler.min.windowmemory" : 524288,
+    "compiler.ordered.fields" : false,
     "compiler\.parallelism" : 0,
     "compiler.queryplanshape" : "zigzag",
     "compiler.runtime.memory.overhead" : 5,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
index 0d4967a..50927b4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
@@ -41,11 +41,13 @@
     "compiler\.indexonly" : true,
     "compiler\.internal\.sanitycheck" : false,
     "compiler\.joinmemory" : 262144,
+    "compiler\.max\.variable\.occurrences\.inlining" : 128,
     "compiler.min.groupmemory" : 524288,
     "compiler.min.joinmemory" : 524288,
     "compiler\.min\.memory\.allocation" : true,
     "compiler.min.sortmemory" : 524288,
     "compiler.min.windowmemory" : 524288,
+    "compiler.ordered.fields" : false,
     "compiler\.parallelism" : -1,
     "compiler.queryplanshape" : "zigzag",
     "compiler.runtime.memory.overhead" : 5,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
index 0295ab2..1b035a0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
@@ -41,11 +41,13 @@
     "compiler\.indexonly" : true,
     "compiler\.internal\.sanitycheck" : false,
     "compiler\.joinmemory" : 262144,
+    "compiler\.max\.variable\.occurrences\.inlining" : 128,
     "compiler.min.groupmemory" : 524288,
     "compiler.min.joinmemory" : 524288,
     "compiler\.min\.memory\.allocation" : true,
     "compiler.min.sortmemory" : 524288,
     "compiler.min.windowmemory" : 524288,
+    "compiler.ordered.fields" : false,
     "compiler\.parallelism" : 3,
     "compiler.queryplanshape" : "zigzag",
     "compiler.runtime.memory.overhead" : 5,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_slice/array_slice_bracket_notation/array_slice_bracket_notation.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_slice/array_slice_bracket_notation/array_slice_bracket_notation.3.adm
new file mode 100644
index 0000000..0e8c7ab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_slice/array_slice_bracket_notation/array_slice_bracket_notation.3.adm
@@ -0,0 +1 @@
+{ "t1": 2, "t2": [ 2, 3 ], "t3": null, "t4": null, "t5": null, "t6": [ 3 ], "t7": [ 2, 3, 4 ], "t8": [ 2, 3, 4, 5 ], "t9": null, "t10": null, "t11": [ 0, 1, 2, 3, 4, 5 ], "t12": 0, "t13": [ 0, 1, 2, 3 ], "t14": null, "t15": null, "t16": null, "t17": null, "t18": [ [ 1, 2 ] ], "t19": [ [ 1, 2, 3, 4, 5 ] ], "t20": [ [ 0, 1, 2 ] ], "t21": [ [ 0, 1, 2, 3, 4, 5 ] ], "t22": [ [ 0, 1, 2, 3, 4, 5 ] ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/join-queries/join-queries.3.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/join-queries/join-queries.3.plan
index b4985f6..5f3e48d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/join-queries/join-queries.3.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/join-queries/join-queries.3.plan
@@ -10,43 +10,37 @@
         -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
-            project ([$$35]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              assign [$$35] <- [{"l_linenumber": $$37, "l_partkey": $$43, "o_custkey": $$44}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ASSIGN  |PARTITIONED|
-                project ([$$37, $$43, $$44]) [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|
-                    join (eq($$36, $$38)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HYBRID_HASH_JOIN [$$36][$$38]  |PARTITIONED|
-                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
-                        project ([$$37, $$43, $$36]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          select (gt($$l.getField(4), 10)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- STREAM_SELECT  |PARTITIONED|
-                            assign [$$43] <- [$$l.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- ASSIGN  |PARTITIONED|
+            assign [$$35] <- [{"l_linenumber": $$37, "l_partkey": $$43, "o_custkey": $$44}] project: [$$35] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ASSIGN  |PARTITIONED|
+              project ([$$37, $$43, $$44]) [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|
+                  join (eq($$36, $$38)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HYBRID_HASH_JOIN [$$36][$$38]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
+                      select (gt($$l.getField(4), 10)) project: [$$37, $$43, $$36] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- STREAM_SELECT  |PARTITIONED|
+                        assign [$$43] <- [$$l.getField(1)] [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]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            data-scan []<-[$$36, $$37, $$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|
-                                data-scan []<-[$$36, $$37, $$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|
-                                    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 [$$38]  |PARTITIONED|
-                        project ([$$44, $$38]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$44] <- [$$o.getField(1)] [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|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                      assign [$$44] <- [$$o.getField(1)] project: [$$44, $$38] [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]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$38, $$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|
-                              data-scan []<-[$$38, $$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|
\ No newline at end of file
+                              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/cardinality-estimation/join-queries/join-queries.4.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/join-queries/join-queries.4.plan
index 45929f5..7d72bad 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/join-queries/join-queries.4.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/join-queries/join-queries.4.plan
@@ -10,43 +10,37 @@
         -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
-            project ([$$34]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              assign [$$34] <- [{"l_orderkey": $$36, "o_custkey": $$41}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ASSIGN  |PARTITIONED|
-                project ([$$36, $$41]) [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|
-                    join (eq($$36, $$38)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HYBRID_HASH_JOIN [$$36][$$38]  |PARTITIONED|
-                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
-                        project ([$$36]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            assign [$$34] <- [{"l_orderkey": $$36, "o_custkey": $$41}] project: [$$34] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ASSIGN  |PARTITIONED|
+              project ([$$36, $$41]) [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|
+                  join (eq($$36, $$38)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HYBRID_HASH_JOIN [$$36][$$38]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
+                      select (gt($$l.getField(1), 5)) project: [$$36] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- STREAM_SELECT  |PARTITIONED|
+                        project ([$$36, $$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- STREAM_PROJECT  |PARTITIONED|
-                          select (gt($$l.getField(1), 5)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- STREAM_SELECT  |PARTITIONED|
-                            project ([$$36, $$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 []<-[$$36, $$37, $$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|
-                                data-scan []<-[$$36, $$37, $$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|
-                                    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 [$$38]  |PARTITIONED|
-                        project ([$$41, $$38]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$41] <- [$$o.getField(1)] [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|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                      assign [$$41] <- [$$o.getField(1)] project: [$$41, $$38] [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]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$38, $$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|
-                              data-scan []<-[$$38, $$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|
\ No newline at end of file
+                              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/cardinality-estimation/join-queries/join-queries.5.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/join-queries/join-queries.5.plan
index 1c63d86..94fc84c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/join-queries/join-queries.5.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/join-queries/join-queries.5.plan
@@ -2,87 +2,79 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$101]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$101] <- [{"$1": $$111}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$111]) [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|
-            group by ([$$l_partkey := $$115; $$o_orderstatus := $$116; $$c_nationkey := $$117]) decor ([]) {
-                      aggregate [$$111] <- [agg-sql-sum($$114)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- AGGREGATE  |LOCAL|
-                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- SORT_GROUP_BY[$$115, $$116, $$117]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HASH_PARTITION_EXCHANGE [$$115, $$116, $$117]  |PARTITIONED|
-                group by ([$$115 := $$102; $$116 := $$103; $$117 := $$104]) decor ([]) {
-                          aggregate [$$114] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- SORT_GROUP_BY[$$102, $$103, $$104]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$102, $$103, $$104]) [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|
-                        join (eq($$109, $$108)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HYBRID_HASH_JOIN [$$109][$$108]  |PARTITIONED|
-                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- HASH_PARTITION_EXCHANGE [$$109]  |PARTITIONED|
-                            project ([$$102, $$103, $$109]) [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|
-                                join (eq($$105, $$107)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- HYBRID_HASH_JOIN [$$105][$$107]  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- HASH_PARTITION_EXCHANGE [$$105]  |PARTITIONED|
-                                    project ([$$102, $$105]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$101] <- [{"$1": $$111}] project: [$$101] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$111]) [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|
+          group by ([$$l_partkey := $$115; $$o_orderstatus := $$116; $$c_nationkey := $$117]) decor ([]) {
+                    aggregate [$$111] <- [agg-sql-sum($$114)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- SORT_GROUP_BY[$$115, $$116, $$117]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HASH_PARTITION_EXCHANGE [$$115, $$116, $$117]  |PARTITIONED|
+              group by ([$$115 := $$102; $$116 := $$103; $$117 := $$104]) decor ([]) {
+                        aggregate [$$114] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- AGGREGATE  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- SORT_GROUP_BY[$$102, $$103, $$104]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  project ([$$102, $$103, $$104]) [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|
+                      join (eq($$109, $$108)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- HYBRID_HASH_JOIN [$$109][$$108]  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- HASH_PARTITION_EXCHANGE [$$109]  |PARTITIONED|
+                          project ([$$102, $$103, $$109]) [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|
+                              join (eq($$105, $$107)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- HYBRID_HASH_JOIN [$$105][$$107]  |PARTITIONED|
+                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- HASH_PARTITION_EXCHANGE [$$105]  |PARTITIONED|
+                                  assign [$$102] <- [$$l.getField(1)] project: [$$102, $$105] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- ASSIGN  |PARTITIONED|
+                                    project ([$$105, $$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                     -- STREAM_PROJECT  |PARTITIONED|
-                                      assign [$$102] <- [$$l.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- ASSIGN  |PARTITIONED|
-                                        project ([$$105, $$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 []<-[$$105, $$106, $$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|
-                                            data-scan []<-[$$105, $$106, $$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|
-                                                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 [$$107]  |PARTITIONED|
-                                    project ([$$103, $$109, $$107]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      assign [$$103, $$109] <- [$$o.getField(2), $$o.getField(1)] [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]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          data-scan []<-[$$107, $$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 [$$108]  |PARTITIONED|
-                            project ([$$104, $$108]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              assign [$$104] <- [$$c.getField(3)] [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|
                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  data-scan []<-[$$108, $$c] <- tpch.Customer [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$107]  |PARTITIONED|
+                                  assign [$$103, $$109] <- [$$o.getField(2), $$o.getField(1)] project: [$$103, $$109, $$107] [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]
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                      data-scan []<-[$$107, $$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 [$$108]  |PARTITIONED|
+                          assign [$$104] <- [$$c.getField(3)] project: [$$104, $$108] [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]
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              data-scan []<-[$$108, $$c] <- tpch.Customer [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/cardinality-estimation/join-queries/join-queries.6.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/join-queries/join-queries.6.plan
index de5075b..df823ed 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/join-queries/join-queries.6.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/join-queries/join-queries.6.plan
@@ -2,83 +2,77 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$101]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$101] <- [{"$1": $$111}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$111]) [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|
-            group by ([$$l_linenumber := $$115; $$o_orderstatus := $$116; $$c_nationkey := $$117]) decor ([]) {
-                      aggregate [$$111] <- [agg-sql-sum($$114)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- AGGREGATE  |LOCAL|
-                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- SORT_GROUP_BY[$$115, $$116, $$117]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HASH_PARTITION_EXCHANGE [$$115, $$116, $$117]  |PARTITIONED|
-                group by ([$$115 := $$106; $$116 := $$103; $$117 := $$104]) decor ([]) {
-                          aggregate [$$114] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- SORT_GROUP_BY[$$106, $$103, $$104]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$106, $$103, $$104]) [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|
-                        join (eq($$109, $$108)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HYBRID_HASH_JOIN [$$109][$$108]  |PARTITIONED|
-                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- HASH_PARTITION_EXCHANGE [$$109]  |PARTITIONED|
-                            project ([$$106, $$103, $$109]) [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|
-                                join (eq($$105, $$107)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- HYBRID_HASH_JOIN [$$105][$$107]  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- HASH_PARTITION_EXCHANGE [$$105]  |PARTITIONED|
-                                    project ([$$106, $$105]) [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 []<-[$$105, $$106, $$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|
-                                            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 [$$107]  |PARTITIONED|
-                                    project ([$$103, $$109, $$107]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      assign [$$103, $$109] <- [$$o.getField(2), $$o.getField(1)] [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]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          data-scan []<-[$$107, $$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 [$$108]  |PARTITIONED|
-                            project ([$$104, $$108]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              assign [$$104] <- [$$c.getField(3)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ASSIGN  |PARTITIONED|
+    assign [$$101] <- [{"$1": $$111}] project: [$$101] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$111]) [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|
+          group by ([$$l_linenumber := $$115; $$o_orderstatus := $$116; $$c_nationkey := $$117]) decor ([]) {
+                    aggregate [$$111] <- [agg-sql-sum($$114)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- SORT_GROUP_BY[$$115, $$116, $$117]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HASH_PARTITION_EXCHANGE [$$115, $$116, $$117]  |PARTITIONED|
+              group by ([$$115 := $$106; $$116 := $$103; $$117 := $$104]) decor ([]) {
+                        aggregate [$$114] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- AGGREGATE  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- SORT_GROUP_BY[$$106, $$103, $$104]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  project ([$$106, $$103, $$104]) [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|
+                      join (eq($$109, $$108)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- HYBRID_HASH_JOIN [$$109][$$108]  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- HASH_PARTITION_EXCHANGE [$$109]  |PARTITIONED|
+                          project ([$$106, $$103, $$109]) [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|
+                              join (eq($$105, $$107)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- HYBRID_HASH_JOIN [$$105][$$107]  |PARTITIONED|
                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  data-scan []<-[$$108, $$c] <- tpch.Customer [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$105]  |PARTITIONED|
+                                  project ([$$106, $$105]) [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|
-                                      empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                      data-scan []<-[$$105, $$106, $$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|
+                                          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 [$$107]  |PARTITIONED|
+                                  assign [$$103, $$109] <- [$$o.getField(2), $$o.getField(1)] project: [$$103, $$109, $$107] [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]
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      data-scan []<-[$$107, $$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 [$$108]  |PARTITIONED|
+                          assign [$$104] <- [$$c.getField(3)] project: [$$104, $$108] [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]
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              data-scan []<-[$$108, $$c] <- tpch.Customer [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/cardinality-estimation/join-queries/join-queries.7.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/join-queries/join-queries.7.plan
index f04c5ff..a6db1e3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/join-queries/join-queries.7.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/join-queries/join-queries.7.plan
@@ -2,83 +2,77 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$101]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$101] <- [{"$1": $$111}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$111]) [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|
-            group by ([$$l_linenumber := $$115; $$c_nationkey := $$116; $$o_orderstatus := $$117]) decor ([]) {
-                      aggregate [$$111] <- [agg-sql-sum($$114)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- AGGREGATE  |LOCAL|
-                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- SORT_GROUP_BY[$$115, $$116, $$117]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HASH_PARTITION_EXCHANGE [$$115, $$116, $$117]  |PARTITIONED|
-                group by ([$$115 := $$106; $$116 := $$103; $$117 := $$104]) decor ([]) {
-                          aggregate [$$114] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- SORT_GROUP_BY[$$106, $$103, $$104]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$106, $$103, $$104]) [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|
-                        join (eq($$109, $$108)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HYBRID_HASH_JOIN [$$109][$$108]  |PARTITIONED|
-                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- HASH_PARTITION_EXCHANGE [$$109]  |PARTITIONED|
-                            project ([$$106, $$104, $$109]) [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|
-                                join (eq($$105, $$107)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- HYBRID_HASH_JOIN [$$105][$$107]  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- HASH_PARTITION_EXCHANGE [$$105]  |PARTITIONED|
-                                    project ([$$106, $$105]) [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 []<-[$$105, $$106, $$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|
-                                            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 [$$107]  |PARTITIONED|
-                                    project ([$$104, $$109, $$107]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      assign [$$104, $$109] <- [$$o.getField(2), $$o.getField(1)] [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]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          data-scan []<-[$$107, $$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 [$$108]  |PARTITIONED|
-                            project ([$$103, $$108]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              assign [$$103] <- [$$c.getField(3)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ASSIGN  |PARTITIONED|
+    assign [$$101] <- [{"$1": $$111}] project: [$$101] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$111]) [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|
+          group by ([$$l_linenumber := $$115; $$c_nationkey := $$116; $$o_orderstatus := $$117]) decor ([]) {
+                    aggregate [$$111] <- [agg-sql-sum($$114)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- SORT_GROUP_BY[$$115, $$116, $$117]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HASH_PARTITION_EXCHANGE [$$115, $$116, $$117]  |PARTITIONED|
+              group by ([$$115 := $$106; $$116 := $$103; $$117 := $$104]) decor ([]) {
+                        aggregate [$$114] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- AGGREGATE  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- SORT_GROUP_BY[$$106, $$103, $$104]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  project ([$$106, $$103, $$104]) [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|
+                      join (eq($$109, $$108)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- HYBRID_HASH_JOIN [$$109][$$108]  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- HASH_PARTITION_EXCHANGE [$$109]  |PARTITIONED|
+                          project ([$$106, $$104, $$109]) [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|
+                              join (eq($$105, $$107)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- HYBRID_HASH_JOIN [$$105][$$107]  |PARTITIONED|
                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  data-scan []<-[$$108, $$c] <- tpch.Customer [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$105]  |PARTITIONED|
+                                  project ([$$106, $$105]) [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|
-                                      empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                      data-scan []<-[$$105, $$106, $$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|
+                                          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 [$$107]  |PARTITIONED|
+                                  assign [$$104, $$109] <- [$$o.getField(2), $$o.getField(1)] project: [$$104, $$109, $$107] [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]
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      data-scan []<-[$$107, $$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 [$$108]  |PARTITIONED|
+                          assign [$$103] <- [$$c.getField(3)] project: [$$103, $$108] [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]
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              data-scan []<-[$$108, $$c] <- tpch.Customer [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/cardinality-estimation/join-queries/join-queries.8.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/join-queries/join-queries.8.plan
index 1b5610d..83ac5aa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/join-queries/join-queries.8.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/join-queries/join-queries.8.plan
@@ -2,133 +2,119 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$119]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$119] <- [{"n_name": $$n_name, "revenue": $$132}] [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]
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          group by ([$$n_name := $$142]) decor ([]) {
-                    aggregate [$$132] <- [agg-global-sql-sum($$141)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$142]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$142]  |PARTITIONED|
-              group by ([$$142 := $$120]) decor ([]) {
-                        aggregate [$$141] <- [agg-local-sql-sum(numeric-multiply($$139, numeric-subtract(1, $$140)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$120]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$139, $$140, $$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|
-                      join (and(eq($$130, $$128), eq($$131, $$143))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- HYBRID_HASH_JOIN [$$130, $$131][$$128, $$143]  |PARTITIONED|
-                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HASH_PARTITION_EXCHANGE [$$130, $$131]  |PARTITIONED|
-                          project ([$$139, $$140, $$130, $$131]) [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|
-                              join (eq($$133, $$127)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- HYBRID_HASH_JOIN [$$133][$$127]  |PARTITIONED|
-                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
-                                  project ([$$139, $$140, $$131, $$133]) [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|
-                                      join (eq($$125, $$124)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- HYBRID_HASH_JOIN [$$124][$$125]  |PARTITIONED|
-                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- HASH_PARTITION_EXCHANGE [$$124]  |PARTITIONED|
-                                          project ([$$131, $$124]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$119] <- [{"n_name": $$n_name, "revenue": $$132}] project: [$$119] [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]
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        group by ([$$n_name := $$142]) decor ([]) {
+                  aggregate [$$132] <- [agg-global-sql-sum($$141)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$142]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$142]  |PARTITIONED|
+            group by ([$$142 := $$120]) decor ([]) {
+                      aggregate [$$141] <- [agg-local-sql-sum(numeric-multiply($$139, numeric-subtract(1, $$140)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$120]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                project ([$$139, $$140, $$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|
+                    join (and(eq($$130, $$128), eq($$131, $$143))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HYBRID_HASH_JOIN [$$130, $$131][$$128, $$143]  |PARTITIONED|
+                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- HASH_PARTITION_EXCHANGE [$$130, $$131]  |PARTITIONED|
+                        project ([$$139, $$140, $$130, $$131]) [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|
+                            join (eq($$133, $$127)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- HYBRID_HASH_JOIN [$$133][$$127]  |PARTITIONED|
+                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
+                                project ([$$139, $$140, $$131, $$133]) [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|
+                                    join (eq($$125, $$124)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- HYBRID_HASH_JOIN [$$124][$$125]  |PARTITIONED|
+                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- HASH_PARTITION_EXCHANGE [$$124]  |PARTITIONED|
+                                        project ([$$131, $$124]) [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|
+                                            join (eq($$123, $$136)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                            -- HYBRID_HASH_JOIN [$$123][$$136]  |PARTITIONED|
+                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- HASH_PARTITION_EXCHANGE [$$123]  |PARTITIONED|
+                                                assign [$$131] <- [$$c.getField(3)] project: [$$131, $$123] [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]
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    data-scan []<-[$$123, $$c] <- tpch.Customer [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 [$$136]  |PARTITIONED|
+                                                select (and(lt($$121, "1994-01-01"), ge($$121, "1993-01-01"))) project: [$$124, $$136] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                -- STREAM_SELECT  |PARTITIONED|
+                                                  assign [$$136, $$121] <- [$$o.getField(1), $$o.getField(4)] project: [$$124, $$136, $$121] [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]
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      data-scan []<-[$$124, $$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 [$$125]  |PARTITIONED|
+                                        assign [$$140, $$139, $$133] <- [$$l.getField(6), $$l.getField(5), $$l.getField(2)] project: [$$139, $$140, $$133, $$125] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- ASSIGN  |PARTITIONED|
+                                          project ([$$125, $$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|
-                                              join (eq($$123, $$136)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- HYBRID_HASH_JOIN [$$123][$$136]  |PARTITIONED|
-                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- HASH_PARTITION_EXCHANGE [$$123]  |PARTITIONED|
-                                                  project ([$$131, $$123]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    assign [$$131] <- [$$c.getField(3)] [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]
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        data-scan []<-[$$123, $$c] <- tpch.Customer [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 [$$136]  |PARTITIONED|
-                                                  project ([$$124, $$136]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    select (and(lt($$121, "1994-01-01"), ge($$121, "1993-01-01"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                      project ([$$124, $$136, $$121]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        assign [$$136, $$121] <- [$$o.getField(1), $$o.getField(4)] [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]
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            data-scan []<-[$$124, $$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 [$$125]  |PARTITIONED|
-                                          project ([$$139, $$140, $$133, $$125]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            assign [$$140, $$139, $$133] <- [$$l.getField(6), $$l.getField(5), $$l.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- ASSIGN  |PARTITIONED|
-                                              project ([$$125, $$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- STREAM_PROJECT  |PARTITIONED|
+                                              data-scan []<-[$$125, $$126, $$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|
-                                                  data-scan []<-[$$125, $$126, $$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|
-                                                      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 [$$127]  |PARTITIONED|
-                                  project ([$$130, $$127]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    assign [$$130] <- [$$s.getField(3)] [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]
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        data-scan []<-[$$127, $$s] <- tpch.Supplier [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 [$$128, $$143]  |PARTITIONED|
-                          project ([$$120, $$128, $$143]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            assign [$$143, $$120] <- [$$128, $$n.getField(1)] [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|
                               exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                data-scan []<-[$$128, $$n] <- tpch.Nation [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- DATASOURCE_SCAN  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$127]  |PARTITIONED|
+                                assign [$$130] <- [$$s.getField(3)] project: [$$130, $$127] [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]
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                    data-scan []<-[$$127, $$s] <- tpch.Supplier [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 [$$128, $$143]  |PARTITIONED|
+                        assign [$$143, $$120] <- [$$128, $$n.getField(1)] project: [$$120, $$128, $$143] [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]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            data-scan []<-[$$128, $$n] <- tpch.Nation [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/cardinality-estimation/single-collection-queries/single-collection-queries.3.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/single-collection-queries/single-collection-queries.3.plan
index 099a04d..dc9bef3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/single-collection-queries/single-collection-queries.3.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/single-collection-queries/single-collection-queries.3.plan
@@ -2,45 +2,41 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$48]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$48] <- [{"$1": $$51}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$51]) [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|
-            group by ([$$o_custkey := $$54]) decor ([]) {
-                      aggregate [$$51] <- [agg-sql-sum($$53)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- AGGREGATE  |LOCAL|
-                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- SORT_GROUP_BY[$$54]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
-                group by ([$$54 := $$49]) decor ([]) {
-                          aggregate [$$53] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- SORT_GROUP_BY[$$49]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$49]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$48] <- [{"$1": $$51}] project: [$$48] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$51]) [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|
+          group by ([$$o_custkey := $$54]) decor ([]) {
+                    aggregate [$$51] <- [agg-sql-sum($$53)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- SORT_GROUP_BY[$$54]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+              group by ([$$54 := $$49]) decor ([]) {
+                        aggregate [$$53] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- AGGREGATE  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- SORT_GROUP_BY[$$49]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  assign [$$49] <- [$$o.getField(1)] project: [$$49] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ASSIGN  |PARTITIONED|
+                    project ([$$o]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                     -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$49] <- [$$o.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
-                        project ([$$o]) [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 [$$50, $$o] <- index-search("Orders", 0, "Default", "tpch", "Orders", false, false, 1, $$55, 0, false, true, 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|
-                            unnest-map [$$50, $$o] <- index-search("Orders", 0, "Default", "tpch", "Orders", false, false, 1, $$55, 0, false, true, 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 [$$55] <- [2] [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|
\ No newline at end of file
+                            assign [$$55] <- [2] [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/cardinality-estimation/single-collection-queries/single-collection-queries.4.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/single-collection-queries/single-collection-queries.4.plan
index 6521ff1..38c8af4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/single-collection-queries/single-collection-queries.4.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/single-collection-queries/single-collection-queries.4.plan
@@ -2,45 +2,41 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$48]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$48] <- [{"$1": $$51}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$51]) [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|
-            group by ([$$o_orderdate := $$54]) decor ([]) {
-                      aggregate [$$51] <- [agg-sql-sum($$53)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- AGGREGATE  |LOCAL|
-                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- SORT_GROUP_BY[$$54]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
-                group by ([$$54 := $$49]) decor ([]) {
-                          aggregate [$$53] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- SORT_GROUP_BY[$$49]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$49]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$48] <- [{"$1": $$51}] project: [$$48] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$51]) [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|
+          group by ([$$o_orderdate := $$54]) decor ([]) {
+                    aggregate [$$51] <- [agg-sql-sum($$53)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- SORT_GROUP_BY[$$54]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+              group by ([$$54 := $$49]) decor ([]) {
+                        aggregate [$$53] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- AGGREGATE  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- SORT_GROUP_BY[$$49]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  assign [$$49] <- [$$o.getField(4)] project: [$$49] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ASSIGN  |PARTITIONED|
+                    project ([$$o]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                     -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$49] <- [$$o.getField(4)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
-                        project ([$$o]) [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 [$$50, $$o] <- index-search("Orders", 0, "Default", "tpch", "Orders", false, false, 1, $$55, 0, false, true, 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|
-                            unnest-map [$$50, $$o] <- index-search("Orders", 0, "Default", "tpch", "Orders", false, false, 1, $$55, 0, false, true, 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 [$$55] <- [5] [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|
\ No newline at end of file
+                            assign [$$55] <- [5] [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/cardinality-estimation/single-collection-queries/single-collection-queries.5.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/single-collection-queries/single-collection-queries.5.plan
index 96633a5..68bc90e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/single-collection-queries/single-collection-queries.5.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/single-collection-queries/single-collection-queries.5.plan
@@ -2,43 +2,39 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$51]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$51] <- [{"$1": $$55}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$55]) [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|
-            group by ([$$l_orderkey := $$59]) decor ([]) {
-                      aggregate [$$55] <- [agg-sql-sum($$58)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- AGGREGATE  |LOCAL|
-                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- SORT_GROUP_BY[$$59]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HASH_PARTITION_EXCHANGE [$$59]  |PARTITIONED|
-                group by ([$$59 := $$53]) decor ([]) {
-                          aggregate [$$58] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- SORT_GROUP_BY[$$53]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$51] <- [{"$1": $$55}] project: [$$51] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$55]) [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|
+          group by ([$$l_orderkey := $$59]) decor ([]) {
+                    aggregate [$$55] <- [agg-sql-sum($$58)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- SORT_GROUP_BY[$$59]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HASH_PARTITION_EXCHANGE [$$59]  |PARTITIONED|
+              group by ([$$59 := $$53]) decor ([]) {
+                        aggregate [$$58] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- AGGREGATE  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- SORT_GROUP_BY[$$53]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  select (and(gt($$l.getField(4), 10), gt($$l.getField(1), 1))) project: [$$53] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_SELECT  |PARTITIONED|
+                    project ([$$53, $$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                     -- STREAM_PROJECT  |PARTITIONED|
-                      select (and(gt($$l.getField(4), 10), gt($$l.getField(1), 1))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_SELECT  |PARTITIONED|
-                        project ([$$53, $$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 []<-[$$53, $$54, $$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|
-                            data-scan []<-[$$53, $$54, $$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|
-                                empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                            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/cardinality-estimation/single-collection-queries/single-collection-queries.6.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/single-collection-queries/single-collection-queries.6.plan
index 3d72881..9512646 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/single-collection-queries/single-collection-queries.6.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/single-collection-queries/single-collection-queries.6.plan
@@ -2,41 +2,37 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$51]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$51] <- [{"$1": $$55}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$55]) [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|
-            group by ([$$l_orderkey := $$59]) decor ([]) {
-                      aggregate [$$55] <- [agg-sql-sum($$58)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- AGGREGATE  |LOCAL|
-                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- SORT_GROUP_BY[$$59]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HASH_PARTITION_EXCHANGE [$$59]  |PARTITIONED|
-                group by ([$$59 := $$53]) decor ([]) {
-                          aggregate [$$58] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- SORT_GROUP_BY[$$53]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      select (and(le($$54, 4), gt($$l.getField(4), 10))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_SELECT  |PARTITIONED|
+    assign [$$51] <- [{"$1": $$55}] project: [$$51] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$55]) [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|
+          group by ([$$l_orderkey := $$59]) decor ([]) {
+                    aggregate [$$55] <- [agg-sql-sum($$58)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- SORT_GROUP_BY[$$59]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HASH_PARTITION_EXCHANGE [$$59]  |PARTITIONED|
+              group by ([$$59 := $$53]) decor ([]) {
+                        aggregate [$$58] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- AGGREGATE  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- SORT_GROUP_BY[$$53]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  select (and(le($$54, 4), gt($$l.getField(4), 10))) project: [$$53] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_SELECT  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$53, $$54, $$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|
-                          data-scan []<-[$$53, $$54, $$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|
-                              empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                          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/cardinality-estimation/single-collection-queries/single-collection-queries.7.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/single-collection-queries/single-collection-queries.7.plan
index eb85763..927d33d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/single-collection-queries/single-collection-queries.7.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/cardinality-estimation/single-collection-queries/single-collection-queries.7.plan
@@ -2,47 +2,41 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$51]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$51] <- [{"$1": $$55}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$55]) [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|
-            group by ([$$l_partkey := $$59]) decor ([]) {
-                      aggregate [$$55] <- [agg-sql-sum($$58)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- AGGREGATE  |LOCAL|
-                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- SORT_GROUP_BY[$$59]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HASH_PARTITION_EXCHANGE [$$59]  |PARTITIONED|
-                group by ([$$59 := $$52]) decor ([]) {
-                          aggregate [$$58] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- SORT_GROUP_BY[$$52]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$52] <- [$$l.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
-                        project ([$$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          select (and(gt($$53, 1), gt($$54, 4))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- STREAM_SELECT  |PARTITIONED|
-                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              unnest-map [$$53, $$54, $$l] <- index-search("LineItem", 0, "Default", "tpch", "LineItem", false, false, 2, $$60, $$61, 0, true, true, 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 [$$60, $$61] <- [1, 4] [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|
\ No newline at end of file
+    assign [$$51] <- [{"$1": $$55}] project: [$$51] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$55]) [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|
+          group by ([$$l_partkey := $$59]) decor ([]) {
+                    aggregate [$$55] <- [agg-sql-sum($$58)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- SORT_GROUP_BY[$$59]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HASH_PARTITION_EXCHANGE [$$59]  |PARTITIONED|
+              group by ([$$59 := $$52]) decor ([]) {
+                        aggregate [$$58] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- AGGREGATE  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- SORT_GROUP_BY[$$52]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  assign [$$52] <- [$$l.getField(1)] project: [$$52] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ASSIGN  |PARTITIONED|
+                    select (and(gt($$53, 1), gt($$54, 4))) project: [$$l] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- STREAM_SELECT  |PARTITIONED|
+                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        unnest-map [$$53, $$54, $$l] <- index-search("LineItem", 0, "Default", "tpch", "LineItem", false, false, 2, $$60, $$61, 0, true, true, 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 [$$60, $$61] <- [1, 4] [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/cloud_storage/storage-size/datasource-function/collection-exists/result.040.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/cloud_storage/storage-size/datasource-function/collection-exists/result.040.regexjson
new file mode 100644
index 0000000..1e9e352
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/cloud_storage/storage-size/datasource-function/collection-exists/result.040.regexjson
@@ -0,0 +1,3 @@
+{
+    "size": "R{.*}"
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/cloud_storage/storage-size/datasource-function/external-collection/result.002.ignore b/asterixdb/asterix-app/src/test/resources/runtimets/results/cloud_storage/storage-size/datasource-function/external-collection/result.002.ignore
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/cloud_storage/storage-size/datasource-function/external-collection/result.002.ignore
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/cloud_storage/storage-size/datasource-function/index-exists/result.040.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/cloud_storage/storage-size/datasource-function/index-exists/result.040.regexjson
new file mode 100644
index 0000000..1e9e352
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/cloud_storage/storage-size/datasource-function/index-exists/result.040.regexjson
@@ -0,0 +1,3 @@
+{
+    "size": "R{.*}"
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/cloud_storage/storage-size/datasource-function/sanity/result.040.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/cloud_storage/storage-size/datasource-function/sanity/result.040.regexjson
new file mode 100644
index 0000000..5cfd541
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/cloud_storage/storage-size/datasource-function/sanity/result.040.regexjson
@@ -0,0 +1 @@
+{"collectionAndIndexesLargerThanCollectionOnly":true,"collectionOnlyLargerThanIndexOnly":true}
\ No newline at end of file
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 fb2c537..b1e3f8c 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
@@ -10,17 +10,15 @@
         -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select (and(gt($$19, "1"), lt($$19, "3"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                assign [$$19] <- [$$d.getField("a")] [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]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$20, $$d] <- test.ColumnDataset filter on: and(gt($$d.getField("a"), "1"), lt($$d.getField("a"), "3")) range-filter on: and(gt($$d.getField("a"), "1"), lt($$d.getField("a"), "3")) [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|
+            select (and(gt($$19, "1"), lt($$19, "3"))) project: [$$d, $$20] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$19] <- [$$d.getField("a")] [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]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$20, $$d] <- test.ColumnDataset filter on: and(gt($$d.getField("a"), "1"), lt($$d.getField("a"), "3")) range-filter on: and(gt($$d.getField("a"), "1"), lt($$d.getField("a"), "3")) [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/column/filter/001/001.009.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.009.plan
index 5247f18..df0e7cd 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
@@ -10,17 +10,15 @@
         -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select (and(gt($$19, 1), lt($$19, 3))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                assign [$$19] <- [$$d.getField("a")] [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]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$20, $$d] <- test.ColumnDataset filter on: and(gt($$d.getField("a"), 1), lt($$d.getField("a"), 3)) range-filter on: and(gt($$d.getField("a"), 1), lt($$d.getField("a"), 3)) [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|
+            select (and(gt($$19, 1), lt($$19, 3))) project: [$$d, $$20] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$19] <- [$$d.getField("a")] [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]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$20, $$d] <- test.ColumnDataset filter on: and(gt($$d.getField("a"), 1), lt($$d.getField("a"), 3)) range-filter on: and(gt($$d.getField("a"), 1), lt($$d.getField("a"), 3)) [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/column/filter/001/001.011.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.011.plan
index c45186a..ab93b30 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
@@ -10,30 +10,28 @@
         -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$28]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select ($$26) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$28, $$d, $$26]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.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: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- SUBPLAN  |PARTITIONED|
-                    assign [$$29] <- [$$d.getField("array")] [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]
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        data-scan []<-[$$28, $$d] <- test.ColumnDataset filter on: lt(scan-collection($$d.getField("array")), "100") range-filter on: lt(scan-collection($$d.getField("array")), "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|
+            select ($$26) project: [$$d, $$28] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$28, $$d, $$26]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.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: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- SUBPLAN  |PARTITIONED|
+                  assign [$$29] <- [$$d.getField("array")] [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]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$28, $$d] <- test.ColumnDataset filter on: lt(scan-collection($$d.getField("array")), "100") range-filter on: lt(scan-collection($$d.getField("array")), "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|
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 71850ec..a62536b 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
@@ -10,30 +10,28 @@
         -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$28]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select ($$26) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$28, $$d, $$26]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.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: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- SUBPLAN  |PARTITIONED|
-                    assign [$$29] <- [$$d.getField("array")] [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]
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        data-scan []<-[$$28, $$d] <- test.ColumnDataset filter on: lt(scan-collection($$d.getField("array")), 100) range-filter on: lt(scan-collection($$d.getField("array")), 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|
+            select ($$26) project: [$$d, $$28] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$28, $$d, $$26]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.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: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- SUBPLAN  |PARTITIONED|
+                  assign [$$29] <- [$$d.getField("array")] [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]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$28, $$d] <- test.ColumnDataset filter on: lt(scan-collection($$d.getField("array")), 100) range-filter on: lt(scan-collection($$d.getField("array")), 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|
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 a9ae3bb..2700a41 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
@@ -10,30 +10,28 @@
         -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$28]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select ($$26) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$28, $$d, $$26]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.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: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- SUBPLAN  |PARTITIONED|
-                    assign [$$29] <- [$$d.getField("array")] [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]
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        data-scan []<-[$$28, $$d] <- test.ColumnDataset filter on: eq(scan-collection($$d.getField("array")), 100) range-filter on: eq(scan-collection($$d.getField("array")), 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|
+            select ($$26) project: [$$d, $$28] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$28, $$d, $$26]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.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: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- SUBPLAN  |PARTITIONED|
+                  assign [$$29] <- [$$d.getField("array")] [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]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$28, $$d] <- test.ColumnDataset filter on: eq(scan-collection($$d.getField("array")), 100) range-filter on: eq(scan-collection($$d.getField("array")), 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|
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 92a9008..f16fc24 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
@@ -10,30 +10,28 @@
         -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$28]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select ($$26) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$28, $$d, $$26]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.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: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- SUBPLAN  |PARTITIONED|
-                    assign [$$29] <- [$$d.getField("array")] [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]
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        data-scan []<-[$$28, $$d] <- test.ColumnDataset filter on: lt(scan-collection($$d.getField("array")), 100.1) range-filter on: lt(scan-collection($$d.getField("array")), 100.1) [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|
+            select ($$26) project: [$$d, $$28] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$28, $$d, $$26]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.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: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- SUBPLAN  |PARTITIONED|
+                  assign [$$29] <- [$$d.getField("array")] [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]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$28, $$d] <- test.ColumnDataset filter on: lt(scan-collection($$d.getField("array")), 100.1) range-filter on: lt(scan-collection($$d.getField("array")), 100.1) [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/column/filter/001/001.019.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.019.plan
index 4910f04..aab321b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.019.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.019.plan
@@ -12,19 +12,15 @@
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             select (and(gt($$item, 60), lt($$item, 100))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
             -- STREAM_SELECT  |PARTITIONED|
-              project ([$$29, $$item]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                unnest $$item <- scan-collection($$30) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- UNNEST  |PARTITIONED|
-                  project ([$$29, $$30]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$30] <- [$$d.getField("array")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
+              unnest $$item <- scan-collection($$30) project: [$$29, $$item] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- UNNEST  |PARTITIONED|
+                assign [$$30] <- [$$d.getField("array")] project: [$$29, $$30] [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]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$29, $$d] <- test.ColumnDataset project ({array:[any]}) filter on: and(gt(scan-collection($$d.getField("array")), 60), lt(scan-collection($$d.getField("array")), 100)) range-filter on: and(gt(scan-collection($$d.getField("array")), 60), lt(scan-collection($$d.getField("array")), 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|
-                        data-scan []<-[$$29, $$d] <- test.ColumnDataset project ({array:[any]}) filter on: and(gt(scan-collection($$d.getField("array")), 60), lt(scan-collection($$d.getField("array")), 100)) range-filter on: and(gt(scan-collection($$d.getField("array")), 60), lt(scan-collection($$d.getField("array")), 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|
+                        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.021.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.021.plan
index c808095..f759f48 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.021.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.021.plan
@@ -12,19 +12,15 @@
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             select (eq($$item, 100)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
             -- STREAM_SELECT  |PARTITIONED|
-              project ([$$27, $$item]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                unnest $$item <- scan-collection($$28) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- UNNEST  |PARTITIONED|
-                  project ([$$27, $$28]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$28] <- [$$d.getField("array")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
+              unnest $$item <- scan-collection($$28) project: [$$27, $$item] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- UNNEST  |PARTITIONED|
+                assign [$$28] <- [$$d.getField("array")] project: [$$27, $$28] [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]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$27, $$d] <- test.ColumnDataset project ({array:[any]}) filter on: eq(scan-collection($$d.getField("array")), 100) range-filter on: eq(scan-collection($$d.getField("array")), 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|
-                        data-scan []<-[$$27, $$d] <- test.ColumnDataset project ({array:[any]}) filter on: eq(scan-collection($$d.getField("array")), 100) range-filter on: eq(scan-collection($$d.getField("array")), 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|
+                        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.023.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.023.plan
index dac0c41..9f4386d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.023.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.023.plan
@@ -12,19 +12,15 @@
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             select (gt($$item, 10000)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
             -- STREAM_SELECT  |PARTITIONED|
-              project ([$$27, $$item]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                unnest $$item <- scan-collection($$28) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- UNNEST  |PARTITIONED|
-                  project ([$$27, $$28]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$28] <- [$$d.getField("array")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
+              unnest $$item <- scan-collection($$28) project: [$$27, $$item] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- UNNEST  |PARTITIONED|
+                assign [$$28] <- [$$d.getField("array")] project: [$$27, $$28] [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]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$27, $$d] <- test.ColumnDataset project ({array:[any]}) filter on: gt(scan-collection($$d.getField("array")), 10000) range-filter on: gt(scan-collection($$d.getField("array")), 10000) [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|
-                        data-scan []<-[$$27, $$d] <- test.ColumnDataset project ({array:[any]}) filter on: gt(scan-collection($$d.getField("array")), 10000) range-filter on: gt(scan-collection($$d.getField("array")), 10000) [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|
+                        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.025.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.025.plan
index 83ace43..4b9c6ab 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.025.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.025.plan
@@ -2,35 +2,29 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$31]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$31] <- [{"a": $$34, "item": $$item}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$34, $$item]) [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 [$$35(ASC), $$34(ASC), $$item(ASC) ]  |PARTITIONED|
-            order (ASC, $$35) (ASC, $$34) (ASC, $$item) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$35(ASC), $$34(ASC), $$item(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                select (gt($$item, 10)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  project ([$$35, $$34, $$item]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    unnest $$item <- scan-collection($$36) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- UNNEST  |PARTITIONED|
-                      select (eq($$34, "1")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_SELECT  |PARTITIONED|
-                        project ([$$35, $$34, $$36]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$34, $$36] <- [$$d.getField("a"), $$d.getField("array")] [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]
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              data-scan []<-[$$35, $$d] <- test.ColumnDataset project ({a:any,array:[any]}) filter on: and(eq($$d.getField("a"), "1"), gt(scan-collection($$d.getField("array")), 10)) range-filter on: and(eq($$d.getField("a"), "1"), gt(scan-collection($$d.getField("array")), 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|
+    assign [$$31] <- [{"a": $$34, "item": $$item}] project: [$$31] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$34, $$item]) [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 [$$35(ASC), $$34(ASC), $$item(ASC) ]  |PARTITIONED|
+          order (ASC, $$35) (ASC, $$34) (ASC, $$item) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$35(ASC), $$34(ASC), $$item(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              select (gt($$item, 10)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                unnest $$item <- scan-collection($$36) project: [$$35, $$34, $$item] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- UNNEST  |PARTITIONED|
+                  select (eq($$34, "1")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_SELECT  |PARTITIONED|
+                    assign [$$34, $$36] <- [$$d.getField("a"), $$d.getField("array")] project: [$$35, $$34, $$36] [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]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        data-scan []<-[$$35, $$d] <- test.ColumnDataset project ({a:any,array:[any]}) filter on: and(eq($$d.getField("a"), "1"), gt(scan-collection($$d.getField("array")), 10)) range-filter on: and(eq($$d.getField("a"), "1"), gt(scan-collection($$d.getField("array")), 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/column/filter/004/004.006.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/004/004.006.plan
index 9be2ed6..df11815 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/004/004.006.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/004/004.006.plan
@@ -2,33 +2,27 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$47]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$47] <- [{"$1": $$49}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$49] <- [agg-sql-sum($$51)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$51] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              select (gt($$D, " ")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$D]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  unnest $$D <- scan-collection($$50) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- UNNEST  |PARTITIONED|
-                    project ([$$50]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$50] <- [$$C.getField("dates")] [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|
-                            data-scan []<-[$$48, $$C] <- TestYelp.YelpCheckin project ({dates:[any]}) filter on: gt(scan-collection($$C.getField("dates")), " ") range-filter on: gt(scan-collection($$C.getField("dates")), " ") [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|
+    assign [$$47] <- [{"$1": $$49}] project: [$$47] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$49] <- [agg-sql-sum($$51)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$51] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            select (gt($$D, " ")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              unnest $$D <- scan-collection($$50) project: [$$D] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- UNNEST  |PARTITIONED|
+                assign [$$50] <- [$$C.getField("dates")] project: [$$50] [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|
+                      data-scan []<-[$$48, $$C] <- TestYelp.YelpCheckin project ({dates:[any]}) filter on: gt(scan-collection($$C.getField("dates")), " ") range-filter on: gt(scan-collection($$C.getField("dates")), " ") [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/column/filter/004/004.102.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/004/004.102.plan
index a246dba..72555d5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/004/004.102.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/004/004.102.plan
@@ -2,33 +2,27 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$48]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$48] <- [{"$1": $$50}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$50] <- [agg-sql-sum($$52)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$52] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              select (eq(substring($$D, 0, 4), "2011")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$D]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  unnest $$D <- scan-collection($$51) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- UNNEST  |PARTITIONED|
-                    project ([$$51]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$51] <- [$$C.getField("dates")] [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|
-                            data-scan []<-[$$49, $$C] <- TestYelp.YelpCheckin project ({dates:[any]}) filter on: eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2011") [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|
+    assign [$$48] <- [{"$1": $$50}] project: [$$48] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$50] <- [agg-sql-sum($$52)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$52] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            select (eq(substring($$D, 0, 4), "2011")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              unnest $$D <- scan-collection($$51) project: [$$D] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- UNNEST  |PARTITIONED|
+                assign [$$51] <- [$$C.getField("dates")] project: [$$51] [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|
+                      data-scan []<-[$$49, $$C] <- TestYelp.YelpCheckin project ({dates:[any]}) filter on: eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2011") [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/column/filter/004/004.104.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/004/004.104.adm
index 7ed187a..b01ec3c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/004/004.104.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/004/004.104.adm
@@ -1 +1 @@
-{ "$1": 20 }
+{ "$1": 20 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/004/004.105.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/004/004.105.plan
index dad5f25..dbdd8f4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/004/004.105.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/004/004.105.plan
@@ -2,37 +2,29 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$51]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$51] <- [{"$1": $$54}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$54] <- [agg-sql-sum($$57)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$57] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              select (or(eq($$58, "2011"), eq($$58, "2016"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$58]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$58] <- [substring($$D, 0, 4)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$51] <- [{"$1": $$54}] project: [$$51] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$54] <- [agg-sql-sum($$57)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$57] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            select (or(eq($$58, "2011"), eq($$58, "2016"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$58] <- [substring($$D, 0, 4)] project: [$$58] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ASSIGN  |PARTITIONED|
+                unnest $$D <- scan-collection($$55) project: [$$D] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- UNNEST  |PARTITIONED|
+                  assign [$$55] <- [$$C.getField("dates")] project: [$$55] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- ASSIGN  |PARTITIONED|
-                    project ([$$D]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    project ([$$C]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                     -- STREAM_PROJECT  |PARTITIONED|
-                      unnest $$D <- scan-collection($$55) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- UNNEST  |PARTITIONED|
-                        project ([$$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$55] <- [$$C.getField("dates")] [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|
-                                data-scan []<-[$$53, $$C] <- TestYelp.YelpCheckin project ({dates:[any]}) filter on: or(eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2011"), eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2016")) [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  |PARTITIONED|
+                        data-scan []<-[$$53, $$C] <- TestYelp.YelpCheckin project ({dates:[any]}) filter on: or(eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2011"), eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2016")) [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/column/filter/004/004.108.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/004/004.108.plan
index 7807f1e..e3975f1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/004/004.108.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/004/004.108.plan
@@ -2,44 +2,40 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$52] <- [{"$1": $$55}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$55] <- [agg-sql-sum($$58)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$58] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              select ($$45) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$45]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  subplan {
-                            aggregate [$$45] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- AGGREGATE  |LOCAL|
-                              select (or(eq($$59, "2011"), eq($$59, "2016"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_SELECT  |LOCAL|
-                                assign [$$59] <- [substring($$D, 0, 4)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- ASSIGN  |LOCAL|
-                                  unnest $$D <- scan-collection($$56) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- SUBPLAN  |PARTITIONED|
-                    project ([$$56]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$52] <- [{"$1": $$55}] project: [$$52] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$55] <- [agg-sql-sum($$58)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$58] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            select ($$45) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$45]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_PROJECT  |PARTITIONED|
+                subplan {
+                          aggregate [$$45] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- AGGREGATE  |LOCAL|
+                            select (or(eq($$59, "2011"), eq($$59, "2016"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- STREAM_SELECT  |LOCAL|
+                              assign [$$59] <- [substring($$D, 0, 4)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- ASSIGN  |LOCAL|
+                                unnest $$D <- scan-collection($$56) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- SUBPLAN  |PARTITIONED|
+                  assign [$$56] <- [$$C.getField("dates")] project: [$$56] [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|
-                      assign [$$56] <- [$$C.getField("dates")] [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|
+                        data-scan []<-[$$54, $$C] <- TestYelp.YelpCheckin project ({dates:[any]}) filter on: or(eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2011"), eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2016")) [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|
-                            data-scan []<-[$$54, $$C] <- TestYelp.YelpCheckin project ({dates:[any]}) filter on: or(eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2011"), eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2016")) [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|
+                            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/004/004.113.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/004/004.113.plan
index 8129cc0..eb99ff7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/004/004.113.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/004/004.113.plan
@@ -2,39 +2,31 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$54]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$54] <- [{"$1": $$57}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$57] <- [agg-sql-sum($$61)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$61] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              select (or(eq($$62, "2011"), eq($$62, "2016"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$62]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$62] <- [substring($$D, 0, 4)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ASSIGN  |PARTITIONED|
-                    project ([$$D]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      unnest $$D <- scan-collection($$59) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- UNNEST  |PARTITIONED|
-                        project ([$$59]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          select (starts-with($$C.getField("business_id"), "-0")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- STREAM_SELECT  |PARTITIONED|
-                            assign [$$59] <- [$$C.getField("dates")] [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|
-                                  data-scan []<-[$$56, $$C] <- TestYelp.YelpCheckin project ({dates:[any],business_id:any}) filter on: and(starts-with($$C.getField("business_id"), "-0"), or(eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2011"), eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2016"))) [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|
+    assign [$$54] <- [{"$1": $$57}] project: [$$54] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$57] <- [agg-sql-sum($$61)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$61] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            select (or(eq($$62, "2011"), eq($$62, "2016"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$62] <- [substring($$D, 0, 4)] project: [$$62] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ASSIGN  |PARTITIONED|
+                unnest $$D <- scan-collection($$59) project: [$$D] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- UNNEST  |PARTITIONED|
+                  select (starts-with($$C.getField("business_id"), "-0")) project: [$$59] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_SELECT  |PARTITIONED|
+                    assign [$$59] <- [$$C.getField("dates")] [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|
+                          data-scan []<-[$$56, $$C] <- TestYelp.YelpCheckin project ({dates:[any],business_id:any}) filter on: and(starts-with($$C.getField("business_id"), "-0"), or(eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2011"), eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2016"))) [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/column/filter/005/005.008.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/005/005.008.plan
index d517ee0..3a64cc9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/005/005.008.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/005/005.008.plan
@@ -2,33 +2,27 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$48]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$48] <- [{"$1": $$50}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$50] <- [agg-sql-sum($$52)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$52] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              select (gt(numeric-add($$D, 1), 2018)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$D]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  unnest $$D <- scan-collection($$51) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- UNNEST  |PARTITIONED|
-                    project ([$$51]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$51] <- [$$C.getField("dates")] [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|
-                            data-scan []<-[$$49, $$C] <- TestYelp.YelpCheckin project ({dates:[any]}) filter on: gt(numeric-add(scan-collection($$C.getField("dates")), 1), 2018) [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|
+    assign [$$48] <- [{"$1": $$50}] project: [$$48] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$50] <- [agg-sql-sum($$52)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$52] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            select (gt(numeric-add($$D, 1), 2018)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              unnest $$D <- scan-collection($$51) project: [$$D] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- UNNEST  |PARTITIONED|
+                assign [$$51] <- [$$C.getField("dates")] project: [$$51] [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|
+                      data-scan []<-[$$49, $$C] <- TestYelp.YelpCheckin project ({dates:[any]}) filter on: gt(numeric-add(scan-collection($$C.getField("dates")), 1), 2018) [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/column/filter/005/005.011.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/005/005.011.plan
index 54e5fa0..2f93b7c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/005/005.011.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/005/005.011.plan
@@ -2,33 +2,27 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$51]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$51] <- [{"$1": $$53}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$53] <- [agg-sql-sum($$55)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$55] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              select (or(gt(numeric-add($$D, 1), 2018), eq(substring($$D, 0, 4), "2011"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$D]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  unnest $$D <- scan-collection($$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- UNNEST  |PARTITIONED|
-                    project ([$$54]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$54] <- [$$C.getField("dates")] [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|
-                            data-scan []<-[$$52, $$C] <- TestYelp.YelpCheckin project ({dates:[any]}) filter on: or(gt(numeric-add(scan-collection($$C.getField("dates")), 1), 2018), eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2011")) [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|
+    assign [$$51] <- [{"$1": $$53}] project: [$$51] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$53] <- [agg-sql-sum($$55)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$55] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            select (or(gt(numeric-add($$D, 1), 2018), eq(substring($$D, 0, 4), "2011"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              unnest $$D <- scan-collection($$54) project: [$$D] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- UNNEST  |PARTITIONED|
+                assign [$$54] <- [$$C.getField("dates")] project: [$$54] [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|
+                      data-scan []<-[$$52, $$C] <- TestYelp.YelpCheckin project ({dates:[any]}) filter on: or(gt(numeric-add(scan-collection($$C.getField("dates")), 1), 2018), eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2011")) [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/column/filter/006/006.010.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/006/006.010.plan
index 4c9f618..f18bf14 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/006/006.010.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/006/006.010.plan
@@ -2,44 +2,40 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$51]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$51] <- [{"$1": $$53}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$53] <- [agg-sql-sum($$57)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$57] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              select ($$44) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$44]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  subplan {
-                            aggregate [$$44] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- AGGREGATE  |LOCAL|
-                              select (eq($$55, array: [ 5.1 ])) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_SELECT  |LOCAL|
-                                assign [$$55] <- [$$point.getField("lon")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- ASSIGN  |LOCAL|
-                                  unnest $$point <- scan-collection($$54) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- SUBPLAN  |PARTITIONED|
-                    project ([$$54]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$51] <- [{"$1": $$53}] project: [$$51] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$53] <- [agg-sql-sum($$57)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$57] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            select ($$44) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$44]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_PROJECT  |PARTITIONED|
+                subplan {
+                          aggregate [$$44] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- AGGREGATE  |LOCAL|
+                            select (eq($$55, array: [ 5.1 ])) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- STREAM_SELECT  |LOCAL|
+                              assign [$$55] <- [$$point.getField("lon")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- ASSIGN  |LOCAL|
+                                unnest $$point <- scan-collection($$54) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- SUBPLAN  |PARTITIONED|
+                  assign [$$54] <- [$$c.getField("geo").getField("coordinates")] project: [$$54] [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|
-                      assign [$$54] <- [$$c.getField("geo").getField("coordinates")] [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|
+                        data-scan []<-[$$52, $$c] <- test.ColumnDataset project ({geo:{coordinates:[{lon:any}]}}) [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|
-                            data-scan []<-[$$52, $$c] <- test.ColumnDataset project ({geo:{coordinates:[{lon:any}]}}) [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|
+                            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/007/007.010.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/007/007.010.plan
index 5fa9b56..8fb9238 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/007/007.010.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/007/007.010.plan
@@ -2,25 +2,23 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$37]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$37] <- [{"$1": $$39}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$39] <- [agg-sql-sum($$42)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$42] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              select (array-contains($$C.getField("checkin_times").getField("dates"), "2018-05-25")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |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|
-                    data-scan []<-[$$38, $$C] <- TestYelp.YelpCheckin project ({checkin_times:{dates:any}}) [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|
+    assign [$$37] <- [{"$1": $$39}] project: [$$37] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$39] <- [agg-sql-sum($$42)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$42] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            select (array-contains($$C.getField("checkin_times").getField("dates"), "2018-05-25")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |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|
+                  data-scan []<-[$$38, $$C] <- TestYelp.YelpCheckin project ({checkin_times:{dates:any}}) [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/column/filter/boolean/boolean.011.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/boolean/boolean.011.adm
index 7a91cf0..c455cca 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/boolean/boolean.011.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/boolean/boolean.011.adm
@@ -6,15 +6,13 @@
     -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        project ([$$17]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          select ($$d.getField("myBoolean")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STREAM_SELECT  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              data-scan []<-[$$17, $$d] <- test.ColumnDataset project ({myBoolean:any}) filter on: $$d.getField("myBoolean") range-filter on: $$d.getField("myBoolean") [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|
+        select ($$d.getField("myBoolean")) project: [$$17] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STREAM_SELECT  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            data-scan []<-[$$17, $$d] <- test.ColumnDataset project ({myBoolean:any}) filter on: $$d.getField("myBoolean") range-filter on: $$d.getField("myBoolean") [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/column/filter/boolean/boolean.021.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/boolean/boolean.021.adm
index 69a65e1..92fa24b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/boolean/boolean.021.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/boolean/boolean.021.adm
@@ -6,15 +6,13 @@
     -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        project ([$$18]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          select (not($$d.getField("myBoolean"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STREAM_SELECT  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              data-scan []<-[$$18, $$d] <- test.ColumnDataset project ({myBoolean:any}) filter on: not($$d.getField("myBoolean")) range-filter on: not($$d.getField("myBoolean")) [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|
+        select (not($$d.getField("myBoolean"))) project: [$$18] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STREAM_SELECT  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            data-scan []<-[$$18, $$d] <- test.ColumnDataset project ({myBoolean:any}) filter on: not($$d.getField("myBoolean")) range-filter on: not($$d.getField("myBoolean")) [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/column/filter/boolean/boolean.031.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/boolean/boolean.031.adm
index a6cf89a..547338a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/boolean/boolean.031.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/boolean/boolean.031.adm
@@ -6,15 +6,13 @@
     -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        project ([$$18]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          select (eq($$d.getField("myBoolean"), true)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STREAM_SELECT  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              data-scan []<-[$$18, $$d] <- test.ColumnDataset project ({myBoolean:any}) filter on: eq($$d.getField("myBoolean"), true) range-filter on: eq($$d.getField("myBoolean"), true) [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|
+        select (eq($$d.getField("myBoolean"), true)) project: [$$18] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STREAM_SELECT  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            data-scan []<-[$$18, $$d] <- test.ColumnDataset project ({myBoolean:any}) filter on: eq($$d.getField("myBoolean"), true) range-filter on: eq($$d.getField("myBoolean"), true) [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/column/filter/not-in_every/not-in_every.101.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/not-in_every/not-in_every.101.plan
index 043246e..f6187ca 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/not-in_every/not-in_every.101.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/not-in_every/not-in_every.101.plan
@@ -10,28 +10,26 @@
         -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select ($$18) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$20, $$d, $$18]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  subplan {
-                            aggregate [$$18] <- [empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- AGGREGATE  |LOCAL|
-                              select (not(if-missing-or-null(neq($$21, "1"), false))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_SELECT  |LOCAL|
-                                nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                         } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- SUBPLAN  |PARTITIONED|
-                    assign [$$21] <- [$$d.getField("a")] [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]
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        data-scan []<-[$$20, $$d] <- test.ColumnDataset [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|
+            select ($$18) project: [$$d, $$20] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$20, $$d, $$18]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_PROJECT  |PARTITIONED|
+                subplan {
+                          aggregate [$$18] <- [empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- AGGREGATE  |LOCAL|
+                            select (not(if-missing-or-null(neq($$21, "1"), false))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- STREAM_SELECT  |LOCAL|
+                              nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- SUBPLAN  |PARTITIONED|
+                  assign [$$21] <- [$$d.getField("a")] [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]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$20, $$d] <- test.ColumnDataset [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/column/filter/not-in_every/not-in_every.201.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/not-in_every/not-in_every.201.plan
index bda6268..d4ae3fa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/not-in_every/not-in_every.201.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/not-in_every/not-in_every.201.plan
@@ -6,50 +6,48 @@
     -- STREAM_PROJECT  |PARTITIONED|
       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-        project ([$$d, $$24]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          select ($$18) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STREAM_SELECT  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              group by ([$$24 := $$20]) decor ([$$d]) {
-                        aggregate [$$18] <- [empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          select (not(is-missing($$23))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- STREAM_SELECT  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- PRE_CLUSTERED_GROUP_BY[$$20]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  order (ASC, $$20) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
-                      project ([$$d, $$23, $$20]) [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 (not(if-missing-or-null(neq($$21, $#1), false))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_LOOP  |PARTITIONED|
-                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              assign [$$21] <- [$$d.getField("a")] [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]
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  data-scan []<-[$$20, $$d] <- test.ColumnDataset [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]
-                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                              assign [$$23] <- [true] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ASSIGN  |UNPARTITIONED|
-                                unnest $#1 <- scan-collection(array: [ "1", "2", "3" ]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- UNNEST  |UNPARTITIONED|
-                                  empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+        select ($$18) project: [$$d, $$24] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STREAM_SELECT  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            group by ([$$24 := $$20]) decor ([$$d]) {
+                      aggregate [$$18] <- [empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        select (not(is-missing($$23))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- STREAM_SELECT  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- PRE_CLUSTERED_GROUP_BY[$$20]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                order (ASC, $$20) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
+                    project ([$$d, $$23, $$20]) [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 (not(if-missing-or-null(neq($$21, $#1), false))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_LOOP  |PARTITIONED|
+                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            assign [$$21] <- [$$d.getField("a")] [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]
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                data-scan []<-[$$20, $$d] <- test.ColumnDataset [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]
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            assign [$$23] <- [true] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- ASSIGN  |UNPARTITIONED|
+                              unnest $#1 <- scan-collection(array: [ "1", "2", "3" ]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- UNNEST  |UNPARTITIONED|
+                                empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/not-in_every/not-in_every.301.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/not-in_every/not-in_every.301.plan
index b632b8b..8ea050d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/not-in_every/not-in_every.301.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/not-in_every/not-in_every.301.plan
@@ -10,30 +10,28 @@
         -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$28]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select ($$26) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$28, $$d, $$26]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  subplan {
-                            aggregate [$$26] <- [empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- AGGREGATE  |LOCAL|
-                              select (not(if-missing-or-null(ge($$x, 100), false))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_SELECT  |LOCAL|
-                                unnest $$x <- 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: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- SUBPLAN  |PARTITIONED|
-                    assign [$$29] <- [$$d.getField("array")] [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]
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        data-scan []<-[$$28, $$d] <- test.ColumnDataset [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|
+            select ($$26) project: [$$d, $$28] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$28, $$d, $$26]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_PROJECT  |PARTITIONED|
+                subplan {
+                          aggregate [$$26] <- [empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- AGGREGATE  |LOCAL|
+                            select (not(if-missing-or-null(ge($$x, 100), false))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- STREAM_SELECT  |LOCAL|
+                              unnest $$x <- 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: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- SUBPLAN  |PARTITIONED|
+                  assign [$$29] <- [$$d.getField("array")] [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]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$28, $$d] <- test.ColumnDataset [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/column/filter/sql-compat/sql-compat.013.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/sql-compat/sql-compat.013.plan
index 7c76122..8230f6a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/sql-compat/sql-compat.013.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/sql-compat/sql-compat.013.plan
@@ -2,62 +2,54 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$197]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$197] <- [{"str2": $$str2, "sum:num3:ok": $$201}] [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 [$$str2(ASC) ]  |PARTITIONED|
-          group by ([$$str2 := $$233]) decor ([]) {
-                    aggregate [$$201] <- [agg-global-sql-sum($$232)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$233]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$233]  |PARTITIONED|
-              group by ([$$233 := $$231]) decor ([]) {
-                        aggregate [$$232] <- [agg-local-sql-sum($$195)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$231]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$195, $$231]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$195] <- [double-default-null($$206)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
-                      project ([$$206, $$231]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        select (or($$184, is-unknown($$231))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_SELECT  |PARTITIONED|
-                          project ([$$206, $$231, $$184]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            subplan {
-                                      aggregate [$$184] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- AGGREGATE  |LOCAL|
-                                        select (eq(string-default-null($$210), "sixteen")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- STREAM_SELECT  |LOCAL|
-                                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- SUBPLAN  |PARTITIONED|
-                              assign [$$231] <- [string-default-null($$210)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ASSIGN  |PARTITIONED|
-                                project ([$$210, $$206]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  assign [$$210, $$206] <- [$#4.getField("str2"), $#4.getField("num3")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ASSIGN  |PARTITIONED|
-                                    project ([$#4]) [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 []<-[$$200, $#4] <- test.ColumnDataset project ({str2:any,num3:any}) [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|
+    assign [$$197] <- [{"str2": $$str2, "sum:num3:ok": $$201}] project: [$$197] [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 [$$str2(ASC) ]  |PARTITIONED|
+        group by ([$$str2 := $$233]) decor ([]) {
+                  aggregate [$$201] <- [agg-global-sql-sum($$232)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$233]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$233]  |PARTITIONED|
+            group by ([$$233 := $$231]) decor ([]) {
+                      aggregate [$$232] <- [agg-local-sql-sum($$195)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$231]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                assign [$$195] <- [double-default-null($$206)] project: [$$195, $$231] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  select (or($$184, is-unknown($$231))) project: [$$206, $$231] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_SELECT  |PARTITIONED|
+                    project ([$$206, $$231, $$184]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      subplan {
+                                aggregate [$$184] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- AGGREGATE  |LOCAL|
+                                  select (eq(string-default-null($$210), "sixteen")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- STREAM_SELECT  |LOCAL|
+                                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                             } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- SUBPLAN  |PARTITIONED|
+                        assign [$$231] <- [string-default-null($$210)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- ASSIGN  |PARTITIONED|
+                          assign [$$210, $$206] <- [$#4.getField("str2"), $#4.getField("num3")] project: [$$210, $$206] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- ASSIGN  |PARTITIONED|
+                            project ([$#4]) [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 []<-[$$200, $#4] <- test.ColumnDataset project ({str2:any,num3:any}) [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/column/filter/subplan/subplan.012.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/subplan/subplan.012.plan
index 1449b29..58261e3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/subplan/subplan.012.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/subplan/subplan.012.plan
@@ -23,17 +23,15 @@
                               -- NESTED_TUPLE_SOURCE  |LOCAL|
                    } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
             -- SUBPLAN  |PARTITIONED|
-              project ([$$57]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                assign [$$57] <- [$$p.getField("arrayOrObject")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ASSIGN  |PARTITIONED|
-                  project ([$$p]) [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 []<-[$$52, $$p] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) filter on: or(eq($$p.getField("arrayOrObject").getField("text"), "7"), eq(scan-collection($$p.getField("arrayOrObject")).getField("text"), "1")) range-filter on: or(eq($$p.getField("arrayOrObject").getField("text"), "7"), eq(scan-collection($$p.getField("arrayOrObject")).getField("text"), "1")) [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|
+              assign [$$57] <- [$$p.getField("arrayOrObject")] project: [$$57] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ASSIGN  |PARTITIONED|
+                project ([$$p]) [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 []<-[$$52, $$p] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) filter on: or(eq($$p.getField("arrayOrObject").getField("text"), "7"), eq(scan-collection($$p.getField("arrayOrObject")).getField("text"), "1")) range-filter on: or(eq($$p.getField("arrayOrObject").getField("text"), "7"), eq(scan-collection($$p.getField("arrayOrObject")).getField("text"), "1")) [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/column/filter/subplan/subplan.022.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/subplan/subplan.022.plan
index 8d3fbd4..f8a88fc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/subplan/subplan.022.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/subplan/subplan.022.plan
@@ -23,17 +23,15 @@
                               -- NESTED_TUPLE_SOURCE  |LOCAL|
                    } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
             -- SUBPLAN  |PARTITIONED|
-              project ([$$57]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                assign [$$57] <- [$$p.getField("arrayOrObject")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ASSIGN  |PARTITIONED|
-                  project ([$$p]) [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 []<-[$$52, $$p] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) [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|
+              assign [$$57] <- [$$p.getField("arrayOrObject")] project: [$$57] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ASSIGN  |PARTITIONED|
+                project ([$$p]) [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 []<-[$$52, $$p] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) [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/column/filter/subplan/subplan.032.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/subplan/subplan.032.plan
index d00429a..588a92e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/subplan/subplan.032.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/subplan/subplan.032.plan
@@ -14,65 +14,55 @@
             -- NESTED_LOOP  |PARTITIONED|
               exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  select (eq($$p.getField("arrayOrObject").getField("text"), "7")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_SELECT  |PARTITIONED|
-                    project ([$$p]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$p] <- [$$b2] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
+                select (eq($$p.getField("arrayOrObject").getField("text"), "7")) project: [] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STREAM_SELECT  |PARTITIONED|
+                  assign [$$p] <- [$$b2] project: [$$p] [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]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- REPLICATE  |PARTITIONED|
                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- REPLICATE  |PARTITIONED|
+                          project ([$$b2]) [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|
-                              project ([$$b2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              data-scan []<-[$$68, $$b2] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) filter on: or(eq($$b2.getField("arrayOrObject").getField("text"), "7"), eq(scan-collection($$b2.getField("arrayOrObject")).getField("text"), "1")) range-filter on: or(eq($$b2.getField("arrayOrObject").getField("text"), "7"), eq(scan-collection($$b2.getField("arrayOrObject")).getField("text"), "1")) [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|
-                                  data-scan []<-[$$68, $$b2] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) filter on: or(eq($$b2.getField("arrayOrObject").getField("text"), "7"), eq(scan-collection($$b2.getField("arrayOrObject")).getField("text"), "1")) range-filter on: or(eq($$b2.getField("arrayOrObject").getField("text"), "7"), eq(scan-collection($$b2.getField("arrayOrObject")).getField("text"), "1")) [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|
+                                  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]
               -- BROADCAST_EXCHANGE  |PARTITIONED|
-                project ([]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |UNPARTITIONED|
-                  select (neq($$69, 0)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_SELECT  |UNPARTITIONED|
-                    aggregate [$$69] <- [agg-sum($$75)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |UNPARTITIONED|
-                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        aggregate [$$75] <- [agg-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |PARTITIONED|
-                          select (eq($$ao.getField("text"), "1")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- STREAM_SELECT  |PARTITIONED|
-                            project ([$$ao]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              unnest $$ao <- scan-collection($$72) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- UNNEST  |PARTITIONED|
-                                project ([$$72]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  assign [$$72] <- [$$b2.getField("arrayOrObject")] [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]
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- REPLICATE  |PARTITIONED|
-                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          project ([$$b2]) [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 []<-[$$68, $$b2] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) filter on: or(eq($$b2.getField("arrayOrObject").getField("text"), "7"), eq(scan-collection($$b2.getField("arrayOrObject")).getField("text"), "1")) range-filter on: or(eq($$b2.getField("arrayOrObject").getField("text"), "7"), eq(scan-collection($$b2.getField("arrayOrObject")).getField("text"), "1")) [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|
+                select (neq($$69, 0)) project: [] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STREAM_SELECT  |UNPARTITIONED|
+                  aggregate [$$69] <- [agg-sum($$75)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |UNPARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      aggregate [$$75] <- [agg-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |PARTITIONED|
+                        select (eq($$ao.getField("text"), "1")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- STREAM_SELECT  |PARTITIONED|
+                          unnest $$ao <- scan-collection($$72) project: [$$ao] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- UNNEST  |PARTITIONED|
+                            assign [$$72] <- [$$b2.getField("arrayOrObject")] project: [$$72] [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]
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- REPLICATE  |PARTITIONED|
+                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    project ([$$b2]) [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 []<-[$$68, $$b2] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) filter on: or(eq($$b2.getField("arrayOrObject").getField("text"), "7"), eq(scan-collection($$b2.getField("arrayOrObject")).getField("text"), "1")) range-filter on: or(eq($$b2.getField("arrayOrObject").getField("text"), "7"), eq(scan-collection($$b2.getField("arrayOrObject")).getField("text"), "1")) [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/column/filter/subplan/subplan.042.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/subplan/subplan.042.plan
index 0d2823e..d570b28 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/subplan/subplan.042.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/subplan/subplan.042.plan
@@ -14,30 +14,26 @@
             -- NESTED_LOOP  |PARTITIONED|
               exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$71]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$71] <- [$$p.getField("arrayOrObject").getField("text")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                assign [$$71] <- [$$p.getField("arrayOrObject").getField("text")] project: [$$71] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  assign [$$p] <- [$$b2] project: [$$p] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- ASSIGN  |PARTITIONED|
-                    project ([$$p]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$p] <- [$$b2] [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]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- REPLICATE  |PARTITIONED|
                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- REPLICATE  |PARTITIONED|
+                          project ([$$b2]) [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|
-                              project ([$$b2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
+                              data-scan []<-[$$68, $$b2] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) [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|
-                                  data-scan []<-[$$68, $$b2] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) [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|
+                                  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]
               -- BROADCAST_EXCHANGE  |PARTITIONED|
                 aggregate [$$69] <- [agg-sum($$75)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
@@ -48,27 +44,23 @@
                     -- AGGREGATE  |PARTITIONED|
                       select (eq($$ao.getField("text"), "1")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- STREAM_SELECT  |PARTITIONED|
-                        project ([$$ao]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          unnest $$ao <- scan-collection($$72) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- UNNEST  |PARTITIONED|
-                            project ([$$72]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              assign [$$72] <- [$$b2.getField("arrayOrObject")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ASSIGN  |PARTITIONED|
+                        unnest $$ao <- scan-collection($$72) project: [$$ao] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- UNNEST  |PARTITIONED|
+                          assign [$$72] <- [$$b2.getField("arrayOrObject")] project: [$$72] [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]
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- REPLICATE  |PARTITIONED|
                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- REPLICATE  |PARTITIONED|
+                                  project ([$$b2]) [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|
-                                      project ([$$b2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                      data-scan []<-[$$68, $$b2] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) [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|
-                                          data-scan []<-[$$68, $$b2] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) [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|
+                                          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/subplan/subplan.052.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/subplan/subplan.052.plan
index d70c8e9..dfe7f62 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/subplan/subplan.052.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/subplan/subplan.052.plan
@@ -23,17 +23,15 @@
                               -- NESTED_TUPLE_SOURCE  |LOCAL|
                    } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
             -- SUBPLAN  |PARTITIONED|
-              project ([$$68]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                assign [$$68] <- [$$p.getField("arrayOrObject")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ASSIGN  |PARTITIONED|
-                  project ([$$p]) [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 []<-[$$66, $$p] <- test.ColumnDataset project ({arrayOrObject:[{text:any}]}) [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|
+              assign [$$68] <- [$$p.getField("arrayOrObject")] project: [$$68] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ASSIGN  |PARTITIONED|
+                project ([$$p]) [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 []<-[$$66, $$p] <- test.ColumnDataset project ({arrayOrObject:[{text:any}]}) [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/column/filter/subplan/subplan.062.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/subplan/subplan.062.plan
index 1c777bd..e8656fe 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/subplan/subplan.062.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/subplan/subplan.062.plan
@@ -6,30 +6,26 @@
     -- NESTED_LOOP  |PARTITIONED|
       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        project ([]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          select (eq($$p.getField("arrayOrObject").getField("text"), "7")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STREAM_SELECT  |PARTITIONED|
-            project ([$$p]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              assign [$$p] <- [$$p2] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ASSIGN  |PARTITIONED|
+        select (eq($$p.getField("arrayOrObject").getField("text"), "7")) project: [] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STREAM_SELECT  |PARTITIONED|
+          assign [$$p] <- [$$p2] project: [$$p] [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]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- REPLICATE  |PARTITIONED|
                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- REPLICATE  |PARTITIONED|
+                  project ([$$p2]) [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|
-                      project ([$$p2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
+                      data-scan []<-[$$62, $$p2] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) filter on: or(eq($$p2.getField("arrayOrObject").getField("text"), "7"), and(ge(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "1"), le(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "2"))) range-filter on: or(eq($$p2.getField("arrayOrObject").getField("text"), "7"), and(ge(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "1"), le(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "2"))) [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|
-                          data-scan []<-[$$62, $$p2] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) filter on: or(eq($$p2.getField("arrayOrObject").getField("text"), "7"), and(ge(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "1"), le(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "2"))) range-filter on: or(eq($$p2.getField("arrayOrObject").getField("text"), "7"), and(ge(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "1"), le(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "2"))) [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|
+                          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]
       -- BROADCAST_EXCHANGE  |PARTITIONED|
         aggregate [$$63] <- [agg-sql-sum($$67)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
@@ -40,31 +36,25 @@
             -- AGGREGATE  |PARTITIONED|
               select (and(ge($$60, "1"), le($$60, "2"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
               -- STREAM_SELECT  |PARTITIONED|
-                project ([$$60]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$60] <- [$$ao.getField("text")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ASSIGN  |PARTITIONED|
-                    project ([$$ao]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      unnest $$ao <- scan-collection($$65) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- UNNEST  |PARTITIONED|
-                        project ([$$65]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$65] <- [$$p2.getField("arrayOrObject")] [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]
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- REPLICATE  |PARTITIONED|
-                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  project ([$$p2]) [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 []<-[$$62, $$p2] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) filter on: or(eq($$p2.getField("arrayOrObject").getField("text"), "7"), and(ge(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "1"), le(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "2"))) range-filter on: or(eq($$p2.getField("arrayOrObject").getField("text"), "7"), and(ge(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "1"), le(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "2"))) [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|
+                assign [$$60] <- [$$ao.getField("text")] project: [$$60] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  unnest $$ao <- scan-collection($$65) project: [$$ao] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- UNNEST  |PARTITIONED|
+                    assign [$$65] <- [$$p2.getField("arrayOrObject")] project: [$$65] [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]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- REPLICATE  |PARTITIONED|
+                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            project ([$$p2]) [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 []<-[$$62, $$p2] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) filter on: or(eq($$p2.getField("arrayOrObject").getField("text"), "7"), and(ge(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "1"), le(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "2"))) range-filter on: or(eq($$p2.getField("arrayOrObject").getField("text"), "7"), and(ge(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "1"), le(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "2"))) [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/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 1e76833..1113862 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
@@ -2,27 +2,23 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$18]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$18] <- [{"display_url": $$22}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$22]) [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 [$$20(ASC) ]  |PARTITIONED|
-            order (ASC, $$20) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$22, $$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$22] <- [get-item($$p.getField("entities").getField("urls"), 0).getField("display_url")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ASSIGN  |PARTITIONED|
+    assign [$$18] <- [{"display_url": $$22}] project: [$$18] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$22]) [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 [$$20(ASC) ]  |PARTITIONED|
+          order (ASC, $$20) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$22] <- [get-item($$p.getField("entities").getField("urls"), 0).getField("display_url")] project: [$$22, $$20] [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]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$20, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [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|
-                      data-scan []<-[$$20, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [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|
+                      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 90f682e..a98b4d0 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
@@ -2,33 +2,27 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$22]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$22] <- [{"display_url": $$27}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$27]) [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 [$$25(ASC) ]  |PARTITIONED|
-            order (ASC, $$25) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$27, $$25]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$27] <- [array-star($$24).getField("display_url")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$22] <- [{"display_url": $$27}] project: [$$22] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$27]) [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 [$$25(ASC) ]  |PARTITIONED|
+          order (ASC, $$25) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$27] <- [array-star($$24).getField("display_url")] project: [$$27, $$25] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ASSIGN  |PARTITIONED|
+                select (not(is-missing($$24))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STREAM_SELECT  |PARTITIONED|
+                  assign [$$24] <- [$$p.getField("entities").getField("urls")] project: [$$25, $$24] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- ASSIGN  |PARTITIONED|
-                    select (not(is-missing($$24))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_SELECT  |PARTITIONED|
-                      project ([$$25, $$24]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$24] <- [$$p.getField("entities").getField("urls")] [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]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            data-scan []<-[$$25, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [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  |PARTITIONED|
+                      data-scan []<-[$$25, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [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/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 128b55e..29b623c 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
@@ -2,31 +2,25 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$28]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$28] <- [{"display_url": $$urls.getField("display_url")}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$urls]) [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 [$$30(ASC) ]  |PARTITIONED|
-            order (ASC, $$30) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$urls, $$30]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  unnest $$urls <- scan-collection($$31) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- UNNEST  |PARTITIONED|
-                    project ([$$30, $$31]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$31] <- [$$p.getField("entities").getField("urls")] [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]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$30, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [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|
+    assign [$$28] <- [{"display_url": $$urls.getField("display_url")}] project: [$$28] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$urls]) [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 [$$30(ASC) ]  |PARTITIONED|
+          order (ASC, $$30) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              unnest $$urls <- scan-collection($$31) project: [$$urls, $$30] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- UNNEST  |PARTITIONED|
+                assign [$$31] <- [$$p.getField("entities").getField("urls")] project: [$$30, $$31] [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]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$30, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [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/column/pushdown/array-access-pushdown/array-access-pushdown.010.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/array-access-pushdown/array-access-pushdown.010.plan
index 7783b10..21ad053 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/array-access-pushdown/array-access-pushdown.010.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/array-access-pushdown/array-access-pushdown.010.plan
@@ -25,17 +25,15 @@
                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                      } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
               -- SUBPLAN  |PARTITIONED|
-                project ([$$51]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$51] <- [$$p.getField("entities").getField("urls")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ASSIGN  |PARTITIONED|
-                    project ([$$p]) [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 []<-[$$49, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [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|
+                assign [$$51] <- [$$p.getField("entities").getField("urls")] project: [$$51] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$p]) [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 []<-[$$49, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [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/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 e38d32f..1b1b9c0 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
@@ -2,27 +2,23 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$19]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$19] <- [get-item(get-item(get-item($$25.getField("coordinates"), 0), 0), 0)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$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]
-          -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
-            order (ASC, $$21) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$25, $$21]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$25] <- [$$p.getField("place").getField("bounding_box")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ASSIGN  |PARTITIONED|
+    assign [$$19] <- [get-item(get-item(get-item($$25.getField("coordinates"), 0), 0), 0)] project: [$$19] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$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]
+        -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+          order (ASC, $$21) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$25] <- [$$p.getField("place").getField("bounding_box")] project: [$$25, $$21] [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]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$21, $$p] <- test.ColumnDataset project ({place:{bounding_box:{coordinates:[[[any]]]}}}) [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|
-                      data-scan []<-[$$21, $$p] <- test.ColumnDataset project ({place:{bounding_box:{coordinates:[[[any]]]}}}) [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|
+                      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 292807e..8790751 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
@@ -2,39 +2,37 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$30]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$30] <- [{"p1": $$p1, "id": $$34}] [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 [$$34(ASC) ]  |PARTITIONED|
-          order (ASC, $$34) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$p1, $$34]) [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|
-                  join (eq($$33, $$34)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HYBRID_HASH_JOIN [$$33][$$34]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HASH_PARTITION_EXCHANGE [$$33]  |PARTITIONED|
-                      data-scan []<-[$$33, $$p1] <- test.ColumnDataset1 [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 [$$34]  |PARTITIONED|
-                      project ([$$34]) [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 []<-[$$34, $$p2] <- test.ColumnDataset2 project ({}) [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|
+    assign [$$30] <- [{"p1": $$p1, "id": $$34}] project: [$$30] [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 [$$34(ASC) ]  |PARTITIONED|
+        order (ASC, $$34) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            project ([$$p1, $$34]) [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|
+                join (eq($$33, $$34)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HYBRID_HASH_JOIN [$$33][$$34]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HASH_PARTITION_EXCHANGE [$$33]  |PARTITIONED|
+                    data-scan []<-[$$33, $$p1] <- test.ColumnDataset1 [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 [$$34]  |PARTITIONED|
+                    project ([$$34]) [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 []<-[$$34, $$p2] <- test.ColumnDataset2 project ({}) [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/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 fe7b07c..5bca144 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
@@ -2,49 +2,43 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$31]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$31] <- [{"age": $$38, "name": $$39}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$38, $$39]) [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 [$$34(ASC) ]  |PARTITIONED|
-            order (ASC, $$34) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$38, $$39, $$34]) [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|
-                    join (eq($$33, $$34)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HYBRID_HASH_JOIN [$$33][$$34]  |PARTITIONED|
-                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- HASH_PARTITION_EXCHANGE [$$33]  |PARTITIONED|
-                        project ([$$38, $$33]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$38] <- [$$p1.getField("age")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- ASSIGN  |PARTITIONED|
+    assign [$$31] <- [{"age": $$38, "name": $$39}] project: [$$31] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$38, $$39]) [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 [$$34(ASC) ]  |PARTITIONED|
+          order (ASC, $$34) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              project ([$$38, $$39, $$34]) [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|
+                  join (eq($$33, $$34)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HYBRID_HASH_JOIN [$$33][$$34]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$33]  |PARTITIONED|
+                      assign [$$38] <- [$$p1.getField("age")] project: [$$38, $$33] [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]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$33, $$p1] <- test.ColumnDataset1 project ({age:any}) [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|
-                              data-scan []<-[$$33, $$p1] <- test.ColumnDataset1 project ({age:any}) [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 [$$34]  |PARTITIONED|
-                        project ([$$39, $$34]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$39] <- [$$p2.getField("name")] [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|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
+                      assign [$$39] <- [$$p2.getField("name")] project: [$$39, $$34] [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]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$34, $$p2] <- test.ColumnDataset3 project ({name:any}) [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|
-                              data-scan []<-[$$34, $$p2] <- test.ColumnDataset3 project ({name:any}) [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|
+                              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.010.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/field-access-pushdown/field-access-pushdown.010.plan
index 75047d2..cce610d 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
@@ -2,25 +2,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$18]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$18] <- [{"id": $$20.getField("id"), "name": $$20.getField("name")}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$18] <- [{"id": $$20.getField("id"), "name": $$20.getField("name")}] project: [$$18] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      assign [$$20] <- [$$p.getField("user")] project: [$$20] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- ASSIGN  |PARTITIONED|
-        project ([$$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        project ([$$p]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
         -- STREAM_PROJECT  |PARTITIONED|
-          assign [$$20] <- [$$p.getField("user")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- ASSIGN  |PARTITIONED|
-            project ([$$p]) [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) ]  |PARTITIONED|
+            order (ASC, $$21) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
               exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
-                order (ASC, $$21) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                data-scan []<-[$$21, $$p] <- test.ColumnDataset4 project ({user:{name:any,id:any}}) [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|
-                    data-scan []<-[$$21, $$p] <- test.ColumnDataset4 project ({user:{name:any,id:any}}) [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|
+                    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 498c7a2..9b44790 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
@@ -2,25 +2,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$21]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$21] <- [switch-case(true, is-array($$23), array-star($$23).getField("text"), $$23.getField("text"))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$21] <- [switch-case(true, is-array($$23), array-star($$23).getField("text"), $$23.getField("text"))] project: [$$21] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      assign [$$23] <- [$$p.getField("arrayOrObject")] project: [$$23] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- ASSIGN  |PARTITIONED|
-        project ([$$23]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        project ([$$p]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
         -- STREAM_PROJECT  |PARTITIONED|
-          assign [$$23] <- [$$p.getField("arrayOrObject")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- ASSIGN  |PARTITIONED|
-            project ([$$p]) [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 [$$24(ASC) ]  |PARTITIONED|
+            order (ASC, $$24) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
               exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-                order (ASC, $$24) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                data-scan []<-[$$24, $$p] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) [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|
-                    data-scan []<-[$$24, $$p] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) [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|
+                    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/join/condition-pushdown/condition-pushdown.012.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/join/condition-pushdown/condition-pushdown.012.plan
index 5afd641..4a1ce9c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/join/condition-pushdown/condition-pushdown.012.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/join/condition-pushdown/condition-pushdown.012.plan
@@ -2,53 +2,47 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$32]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$32] <- [{"c1": $$40, "c2": $$41}] [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 [$$40(ASC), $$41(ASC) ]  |PARTITIONED|
-          order (ASC, $$40) (ASC, $$41) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$40, $$41]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select (eq($$37, $$t2.getField("c_s"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  assign [$$41] <- [$$t2.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ASSIGN  |PARTITIONED|
-                    project ([$$40, $$37, $$t2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
+    assign [$$32] <- [{"c1": $$40, "c2": $$41}] project: [$$32] [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 [$$40(ASC), $$41(ASC) ]  |PARTITIONED|
+        order (ASC, $$40) (ASC, $$41) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select (eq($$37, $$t2.getField("c_s"))) project: [$$40, $$41] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$41] <- [$$t2.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ASSIGN  |PARTITIONED|
+                project ([$$40, $$37, $$t2]) [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 [$$36, $$t2] <- index-search("TestOpenColumn2", 0, "Default", "test", "TestOpenColumn2", true, false, 1, $$45, 1, $$45, true, true, true) project ({c_s:any,c_x:any}) [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|
-                        unnest-map [$$36, $$t2] <- index-search("TestOpenColumn2", 0, "Default", "test", "TestOpenColumn2", true, false, 1, $$45, 1, $$45, true, true, true) project ({c_s:any,c_x:any}) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- BTREE_SEARCH  |PARTITIONED|
+                        order (ASC, $$45) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
                           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            order (ASC, $$45) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
+                            project ([$$40, $$37, $$45]) [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|
-                                project ([$$40, $$37, $$45]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_PROJECT  |PARTITIONED|
+                                unnest-map [$$44, $$45] <- index-search("idx_column_t2_s", 0, "Default", "test", "TestOpenColumn2", true, true, 1, $$37, 1, $$37, true, true, true) [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|
-                                    unnest-map [$$44, $$45] <- index-search("idx_column_t2_s", 0, "Default", "test", "TestOpenColumn2", true, true, 1, $$37, 1, $$37, true, true, true) [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]
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        project ([$$37, $$40]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          assign [$$37, $$40] <- [to-string($$t1.getField("c_s")), $$t1.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- ASSIGN  |PARTITIONED|
-                                            project ([$$t1]) [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 []<-[$$35, $$t1] <- test.TestOpenColumn1 project ({c_s:any,c_x:any}) [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|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    assign [$$37, $$40] <- [to-string($$t1.getField("c_s")), $$t1.getField("c_x")] project: [$$37, $$40] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- ASSIGN  |PARTITIONED|
+                                      project ([$$t1]) [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 []<-[$$35, $$t1] <- test.TestOpenColumn1 project ({c_s:any,c_x:any}) [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/column/pushdown/join/condition-pushdown/condition-pushdown.014.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/join/condition-pushdown/condition-pushdown.014.plan
index 493b14f..b0d6f88 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/join/condition-pushdown/condition-pushdown.014.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/join/condition-pushdown/condition-pushdown.014.plan
@@ -2,59 +2,53 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$37]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$37] <- [{"c1": $$47, "c2": $$48}] [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 [$$47(ASC), $$48(ASC) ]  |PARTITIONED|
-          order (ASC, $$47) (ASC, $$48) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$47(ASC), $$48(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$47, $$48]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select (eq($$42, $$t2.getField("c_s"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  select (eq($$t2.getField("c_i64"), 2.25)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_SELECT  |PARTITIONED|
-                    assign [$$48] <- [$$t2.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
-                      project ([$$47, $$42, $$t2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
+    assign [$$37] <- [{"c1": $$47, "c2": $$48}] project: [$$37] [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 [$$47(ASC), $$48(ASC) ]  |PARTITIONED|
+        order (ASC, $$47) (ASC, $$48) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$47(ASC), $$48(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select (eq($$42, $$t2.getField("c_s"))) project: [$$47, $$48] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              select (eq($$t2.getField("c_i64"), 2.25)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                assign [$$48] <- [$$t2.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$47, $$42, $$t2]) [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 [$$41, $$t2] <- index-search("TestOpenColumn2", 0, "Default", "test", "TestOpenColumn2", true, false, 1, $$52, 1, $$52, true, true, true) project ({c_i64:any,c_s:any,c_x:any}) range-filter on: eq($$t2.getField("c_i64"), 2.25) [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|
-                          unnest-map [$$41, $$t2] <- index-search("TestOpenColumn2", 0, "Default", "test", "TestOpenColumn2", true, false, 1, $$52, 1, $$52, true, true, true) project ({c_i64:any,c_s:any,c_x:any}) range-filter on: eq($$t2.getField("c_i64"), 2.25) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- BTREE_SEARCH  |PARTITIONED|
+                          order (ASC, $$52) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
                             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              order (ASC, $$52) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
+                              project ([$$47, $$42, $$52]) [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|
-                                  project ([$$47, $$42, $$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  unnest-map [$$51, $$52] <- index-search("idx_column_t2_s", 0, "Default", "test", "TestOpenColumn2", true, true, 1, $$42, 1, $$42, true, true, true) [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|
-                                      unnest-map [$$51, $$52] <- index-search("idx_column_t2_s", 0, "Default", "test", "TestOpenColumn2", true, true, 1, $$42, 1, $$42, true, true, true) [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]
-                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                          project ([$$47, $$42]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            assign [$$42] <- [to-string($$t1.getField("c_s"))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- ASSIGN  |PARTITIONED|
-                                              select (eq($$t1.getField("c_i64"), 2)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                assign [$$47] <- [$$t1.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- ASSIGN  |PARTITIONED|
-                                                  project ([$$t1]) [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 []<-[$$40, $$t1] <- test.TestOpenColumn1 project ({c_i64:any,c_s:any,c_x:any}) filter on: eq($$t1.getField("c_i64"), 2) range-filter on: eq($$t1.getField("c_i64"), 2) [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|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      assign [$$42] <- [to-string($$t1.getField("c_s"))] project: [$$47, $$42] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- ASSIGN  |PARTITIONED|
+                                        select (eq($$t1.getField("c_i64"), 2)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          assign [$$47] <- [$$t1.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- ASSIGN  |PARTITIONED|
+                                            project ([$$t1]) [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 []<-[$$40, $$t1] <- test.TestOpenColumn1 project ({c_i64:any,c_s:any,c_x:any}) filter on: eq($$t1.getField("c_i64"), 2) range-filter on: eq($$t1.getField("c_i64"), 2) [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/column/pushdown/join/condition-pushdown/condition-pushdown.021.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/join/condition-pushdown/condition-pushdown.021.plan
index 9a42877..a557644 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/join/condition-pushdown/condition-pushdown.021.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/join/condition-pushdown/condition-pushdown.021.plan
@@ -2,53 +2,47 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$32]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$32] <- [{"c1": $$40, "c2": $$41}] [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 [$$40(ASC), $$41(ASC) ]  |PARTITIONED|
-          order (ASC, $$40) (ASC, $$41) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$40, $$41]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select (eq($$37, $$t2.getField("c_s"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  assign [$$41] <- [$$t2.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ASSIGN  |PARTITIONED|
-                    project ([$$40, $$37, $$t2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
+    assign [$$32] <- [{"c1": $$40, "c2": $$41}] project: [$$32] [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 [$$40(ASC), $$41(ASC) ]  |PARTITIONED|
+        order (ASC, $$40) (ASC, $$41) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select (eq($$37, $$t2.getField("c_s"))) project: [$$40, $$41] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$41] <- [$$t2.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ASSIGN  |PARTITIONED|
+                project ([$$40, $$37, $$t2]) [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 [$$36, $$t2] <- index-search("TestOpenRow2", 0, "Default", "test", "TestOpenRow2", true, false, 1, $$45, 1, $$45, true, true, true) [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|
-                        unnest-map [$$36, $$t2] <- index-search("TestOpenRow2", 0, "Default", "test", "TestOpenRow2", true, false, 1, $$45, 1, $$45, true, true, true) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- BTREE_SEARCH  |PARTITIONED|
+                        order (ASC, $$45) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
                           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            order (ASC, $$45) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
+                            project ([$$40, $$37, $$45]) [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|
-                                project ([$$40, $$37, $$45]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_PROJECT  |PARTITIONED|
+                                unnest-map [$$44, $$45] <- index-search("idx_row_t2_s", 0, "Default", "test", "TestOpenRow2", true, true, 1, $$37, 1, $$37, true, true, true) [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|
-                                    unnest-map [$$44, $$45] <- index-search("idx_row_t2_s", 0, "Default", "test", "TestOpenRow2", true, true, 1, $$37, 1, $$37, true, true, true) [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]
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        project ([$$37, $$40]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          assign [$$37, $$40] <- [to-string($$t1.getField("c_s")), $$t1.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- ASSIGN  |PARTITIONED|
-                                            project ([$$t1]) [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 []<-[$$35, $$t1] <- test.TestOpenColumn1 project ({c_s:any,c_x:any}) [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|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    assign [$$37, $$40] <- [to-string($$t1.getField("c_s")), $$t1.getField("c_x")] project: [$$37, $$40] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- ASSIGN  |PARTITIONED|
+                                      project ([$$t1]) [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 []<-[$$35, $$t1] <- test.TestOpenColumn1 project ({c_s:any,c_x:any}) [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/column/pushdown/join/condition-pushdown/condition-pushdown.023.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/join/condition-pushdown/condition-pushdown.023.plan
index 3d11eab..73fb237 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/join/condition-pushdown/condition-pushdown.023.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/join/condition-pushdown/condition-pushdown.023.plan
@@ -2,59 +2,53 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$37]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$37] <- [{"c1": $$47, "c2": $$48}] [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 [$$47(ASC), $$48(ASC) ]  |PARTITIONED|
-          order (ASC, $$47) (ASC, $$48) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$47(ASC), $$48(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$47, $$48]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select (eq($$42, $$t2.getField("c_s"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  select (eq($$t2.getField("c_i64"), 2.25)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_SELECT  |PARTITIONED|
-                    assign [$$48] <- [$$t2.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
-                      project ([$$47, $$42, $$t2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
+    assign [$$37] <- [{"c1": $$47, "c2": $$48}] project: [$$37] [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 [$$47(ASC), $$48(ASC) ]  |PARTITIONED|
+        order (ASC, $$47) (ASC, $$48) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$47(ASC), $$48(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select (eq($$42, $$t2.getField("c_s"))) project: [$$47, $$48] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              select (eq($$t2.getField("c_i64"), 2.25)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                assign [$$48] <- [$$t2.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$47, $$42, $$t2]) [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 [$$41, $$t2] <- index-search("TestOpenRow2", 0, "Default", "test", "TestOpenRow2", true, false, 1, $$52, 1, $$52, true, true, true) [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|
-                          unnest-map [$$41, $$t2] <- index-search("TestOpenRow2", 0, "Default", "test", "TestOpenRow2", true, false, 1, $$52, 1, $$52, true, true, true) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- BTREE_SEARCH  |PARTITIONED|
+                          order (ASC, $$52) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
                             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              order (ASC, $$52) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
+                              project ([$$47, $$42, $$52]) [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|
-                                  project ([$$47, $$42, $$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  unnest-map [$$51, $$52] <- index-search("idx_row_t2_s", 0, "Default", "test", "TestOpenRow2", true, true, 1, $$42, 1, $$42, true, true, true) [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|
-                                      unnest-map [$$51, $$52] <- index-search("idx_row_t2_s", 0, "Default", "test", "TestOpenRow2", true, true, 1, $$42, 1, $$42, true, true, true) [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]
-                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                          project ([$$47, $$42]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            assign [$$42] <- [to-string($$t1.getField("c_s"))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- ASSIGN  |PARTITIONED|
-                                              select (eq($$t1.getField("c_i64"), 2)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                assign [$$47] <- [$$t1.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- ASSIGN  |PARTITIONED|
-                                                  project ([$$t1]) [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 []<-[$$40, $$t1] <- test.TestOpenColumn1 project ({c_i64:any,c_s:any,c_x:any}) filter on: eq($$t1.getField("c_i64"), 2) range-filter on: eq($$t1.getField("c_i64"), 2) [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|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      assign [$$42] <- [to-string($$t1.getField("c_s"))] project: [$$47, $$42] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- ASSIGN  |PARTITIONED|
+                                        select (eq($$t1.getField("c_i64"), 2)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          assign [$$47] <- [$$t1.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- ASSIGN  |PARTITIONED|
+                                            project ([$$t1]) [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 []<-[$$40, $$t1] <- test.TestOpenColumn1 project ({c_i64:any,c_s:any,c_x:any}) filter on: eq($$t1.getField("c_i64"), 2) range-filter on: eq($$t1.getField("c_i64"), 2) [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/column/pushdown/join/condition-pushdown/condition-pushdown.031.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/join/condition-pushdown/condition-pushdown.031.plan
index e7345eb..6bada47 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/join/condition-pushdown/condition-pushdown.031.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/join/condition-pushdown/condition-pushdown.031.plan
@@ -2,53 +2,47 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$32]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$32] <- [{"c1": $$40, "c2": $$41}] [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 [$$40(ASC), $$41(ASC) ]  |PARTITIONED|
-          order (ASC, $$40) (ASC, $$41) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$40, $$41]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select (eq($$37, $$t2.getField("c_s"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  assign [$$41] <- [$$t2.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ASSIGN  |PARTITIONED|
-                    project ([$$40, $$37, $$t2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
+    assign [$$32] <- [{"c1": $$40, "c2": $$41}] project: [$$32] [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 [$$40(ASC), $$41(ASC) ]  |PARTITIONED|
+        order (ASC, $$40) (ASC, $$41) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select (eq($$37, $$t2.getField("c_s"))) project: [$$40, $$41] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$41] <- [$$t2.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ASSIGN  |PARTITIONED|
+                project ([$$40, $$37, $$t2]) [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 [$$36, $$t2] <- index-search("TestOpenColumn2", 0, "Default", "test", "TestOpenColumn2", true, false, 1, $$45, 1, $$45, true, true, true) project ({c_s:any,c_x:any}) [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|
-                        unnest-map [$$36, $$t2] <- index-search("TestOpenColumn2", 0, "Default", "test", "TestOpenColumn2", true, false, 1, $$45, 1, $$45, true, true, true) project ({c_s:any,c_x:any}) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- BTREE_SEARCH  |PARTITIONED|
+                        order (ASC, $$45) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
                           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            order (ASC, $$45) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
+                            project ([$$40, $$37, $$45]) [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|
-                                project ([$$40, $$37, $$45]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_PROJECT  |PARTITIONED|
+                                unnest-map [$$44, $$45] <- index-search("idx_column_t2_s", 0, "Default", "test", "TestOpenColumn2", true, true, 1, $$37, 1, $$37, true, true, true) [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|
-                                    unnest-map [$$44, $$45] <- index-search("idx_column_t2_s", 0, "Default", "test", "TestOpenColumn2", true, true, 1, $$37, 1, $$37, true, true, true) [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]
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        project ([$$37, $$40]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          assign [$$37, $$40] <- [to-string($$t1.getField("c_s")), $$t1.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- ASSIGN  |PARTITIONED|
-                                            project ([$$t1]) [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 []<-[$$35, $$t1] <- test.TestOpenRow1 [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|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    assign [$$37, $$40] <- [to-string($$t1.getField("c_s")), $$t1.getField("c_x")] project: [$$37, $$40] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- ASSIGN  |PARTITIONED|
+                                      project ([$$t1]) [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 []<-[$$35, $$t1] <- test.TestOpenRow1 [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/column/pushdown/join/condition-pushdown/condition-pushdown.033.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/join/condition-pushdown/condition-pushdown.033.plan
index ff129d8..9e5c65e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/join/condition-pushdown/condition-pushdown.033.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/join/condition-pushdown/condition-pushdown.033.plan
@@ -2,59 +2,53 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$37]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$37] <- [{"c1": $$47, "c2": $$48}] [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 [$$47(ASC), $$48(ASC) ]  |PARTITIONED|
-          order (ASC, $$47) (ASC, $$48) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$47(ASC), $$48(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$47, $$48]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select (eq($$42, $$t2.getField("c_s"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  select (eq($$t2.getField("c_i64"), 2.25)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_SELECT  |PARTITIONED|
-                    assign [$$48] <- [$$t2.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
-                      project ([$$47, $$42, $$t2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
+    assign [$$37] <- [{"c1": $$47, "c2": $$48}] project: [$$37] [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 [$$47(ASC), $$48(ASC) ]  |PARTITIONED|
+        order (ASC, $$47) (ASC, $$48) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$47(ASC), $$48(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select (eq($$42, $$t2.getField("c_s"))) project: [$$47, $$48] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              select (eq($$t2.getField("c_i64"), 2.25)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                assign [$$48] <- [$$t2.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$47, $$42, $$t2]) [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 [$$41, $$t2] <- index-search("TestOpenColumn2", 0, "Default", "test", "TestOpenColumn2", true, false, 1, $$52, 1, $$52, true, true, true) project ({c_i64:any,c_s:any,c_x:any}) range-filter on: eq($$t2.getField("c_i64"), 2.25) [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|
-                          unnest-map [$$41, $$t2] <- index-search("TestOpenColumn2", 0, "Default", "test", "TestOpenColumn2", true, false, 1, $$52, 1, $$52, true, true, true) project ({c_i64:any,c_s:any,c_x:any}) range-filter on: eq($$t2.getField("c_i64"), 2.25) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- BTREE_SEARCH  |PARTITIONED|
+                          order (ASC, $$52) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
                             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              order (ASC, $$52) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
+                              project ([$$47, $$42, $$52]) [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|
-                                  project ([$$47, $$42, $$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  unnest-map [$$51, $$52] <- index-search("idx_column_t2_s", 0, "Default", "test", "TestOpenColumn2", true, true, 1, $$42, 1, $$42, true, true, true) [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|
-                                      unnest-map [$$51, $$52] <- index-search("idx_column_t2_s", 0, "Default", "test", "TestOpenColumn2", true, true, 1, $$42, 1, $$42, true, true, true) [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]
-                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                          project ([$$47, $$42]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            assign [$$42] <- [to-string($$t1.getField("c_s"))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- ASSIGN  |PARTITIONED|
-                                              select (eq($$t1.getField("c_i64"), 2)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                assign [$$47] <- [$$t1.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- ASSIGN  |PARTITIONED|
-                                                  project ([$$t1]) [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 []<-[$$40, $$t1] <- test.TestOpenRow1 [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|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      assign [$$42] <- [to-string($$t1.getField("c_s"))] project: [$$47, $$42] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- ASSIGN  |PARTITIONED|
+                                        select (eq($$t1.getField("c_i64"), 2)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          assign [$$47] <- [$$t1.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- ASSIGN  |PARTITIONED|
+                                            project ([$$t1]) [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 []<-[$$40, $$t1] <- test.TestOpenRow1 [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/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 fbb690d..cd1f227 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
@@ -2,39 +2,35 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$61]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$61] <- [{"$1": $$66, "$2": $$67}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$66, $$67] <- [agg-global-sql-sum($$70), agg-global-sql-sum($$71)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$70, $$71] <- [agg-local-sql-sum($$54), agg-local-sql-sum($$59)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              project ([$$54, $$59]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$61] <- [{"$1": $$66, "$2": $$67}] project: [$$61] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$66, $$67] <- [agg-global-sql-sum($$70), agg-global-sql-sum($$71)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$70, $$71] <- [agg-local-sql-sum($$54), agg-local-sql-sum($$59)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            assign [$$59, $$54] <- [object-length($$p2), object-length($$p1)] project: [$$54, $$59] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ASSIGN  |PARTITIONED|
+              project ([$$p1, $$p2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
               -- STREAM_PROJECT  |PARTITIONED|
-                assign [$$59, $$54] <- [object-length($$p2), object-length($$p1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ASSIGN  |PARTITIONED|
-                  project ([$$p1, $$p2]) [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|
+                  join (eq($$63, $$62)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HYBRID_HASH_JOIN [$$62][$$63]  |PARTITIONED|
                     exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      join (eq($$63, $$62)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- HYBRID_HASH_JOIN [$$62][$$63]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                      data-scan []<-[$$62, $$p1] <- test.ColumnDataset [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]
-                        -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
-                          data-scan []<-[$$62, $$p1] <- test.ColumnDataset [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|
+                        -- 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 [$$63]  |PARTITIONED|
+                      data-scan []<-[$$63, $$p2] <- test.RowDataset [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]
-                        -- HASH_PARTITION_EXCHANGE [$$63]  |PARTITIONED|
-                          data-scan []<-[$$63, $$p2] <- test.RowDataset [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|
+                        -- 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.008.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.008.plan
index d88c101..02e6f1b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.008.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.008.plan
@@ -2,51 +2,43 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$69]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$69] <- [{"text": $$text, "$1": $$73}] [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 [$$text(ASC) ]  |PARTITIONED|
-          group by ([$$text := $$81]) decor ([]) {
-                    aggregate [$$73] <- [agg-global-sql-sum($$80)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$81]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$81]  |PARTITIONED|
-              group by ([$$81 := $$71]) decor ([]) {
-                        aggregate [$$80] <- [agg-local-sql-sum(sql-count($$75))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$71]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  select (eq(lowercase($$71), "string")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_SELECT  |PARTITIONED|
-                    project ([$$75, $$71]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$71] <- [$$ht.getField("display_url")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$69] <- [{"text": $$text, "$1": $$73}] project: [$$69] [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 [$$text(ASC) ]  |PARTITIONED|
+        group by ([$$text := $$81]) decor ([]) {
+                  aggregate [$$73] <- [agg-global-sql-sum($$80)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$81]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$81]  |PARTITIONED|
+            group by ([$$81 := $$71]) decor ([]) {
+                      aggregate [$$80] <- [agg-local-sql-sum(sql-count($$75))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$71]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                select (eq(lowercase($$71), "string")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STREAM_SELECT  |PARTITIONED|
+                  assign [$$71] <- [$$ht.getField("display_url")] project: [$$75, $$71] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ASSIGN  |PARTITIONED|
+                    unnest $$ht <- scan-collection($$75) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- UNNEST  |PARTITIONED|
+                      assign [$$75] <- [$$p1.getField("entities").getField("urls")] project: [$$75] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- ASSIGN  |PARTITIONED|
-                        unnest $$ht <- scan-collection($$75) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- UNNEST  |PARTITIONED|
-                          project ([$$75]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            assign [$$75] <- [$$p1.getField("entities").getField("urls")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- ASSIGN  |PARTITIONED|
-                              project ([$$p1]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                select (neq($$72, "0")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    data-scan []<-[$$72, $$p1] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) filter on: eq(lowercase(scan-collection($$p1.getField("entities").getField("urls")).getField("display_url")), "string") [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|
+                        select (neq($$72, "0")) project: [$$p1] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- STREAM_SELECT  |PARTITIONED|
+                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            data-scan []<-[$$72, $$p1] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) filter on: eq(lowercase(scan-collection($$p1.getField("entities").getField("urls")).getField("display_url")), "string") [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/column/pushdown/other-pushdowns/other-pushdowns.010.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.010.plan
index 5a7b0b3..4723a05 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.010.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.010.plan
@@ -2,53 +2,43 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$68]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$68] <- [{"text": $$text, "$1": $$72}] [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 [$$text(ASC) ]  |PARTITIONED|
-          group by ([$$text := $$79]) decor ([]) {
-                    aggregate [$$72] <- [agg-global-sql-sum($$78)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$79]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$79]  |PARTITIONED|
-              group by ([$$79 := $$70]) decor ([]) {
-                        aggregate [$$78] <- [agg-local-sql-sum(sql-sum($$75))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$70]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  select (eq(lowercase($$70), "string")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_SELECT  |PARTITIONED|
-                    project ([$$75, $$70]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$75, $$70] <- [$$ht.getField("indices"), $$ht.getField("display_url")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$68] <- [{"text": $$text, "$1": $$72}] project: [$$68] [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 [$$text(ASC) ]  |PARTITIONED|
+        group by ([$$text := $$79]) decor ([]) {
+                  aggregate [$$72] <- [agg-global-sql-sum($$78)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$79]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$79]  |PARTITIONED|
+            group by ([$$79 := $$70]) decor ([]) {
+                      aggregate [$$78] <- [agg-local-sql-sum(sql-sum($$75))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$70]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                select (eq(lowercase($$70), "string")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STREAM_SELECT  |PARTITIONED|
+                  assign [$$75, $$70] <- [$$ht.getField("indices"), $$ht.getField("display_url")] project: [$$75, $$70] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ASSIGN  |PARTITIONED|
+                    unnest $$ht <- scan-collection($$74) project: [$$ht] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- UNNEST  |PARTITIONED|
+                      assign [$$74] <- [$$p1.getField("entities").getField("urls")] project: [$$74] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- ASSIGN  |PARTITIONED|
-                        project ([$$ht]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          unnest $$ht <- scan-collection($$74) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- UNNEST  |PARTITIONED|
-                            project ([$$74]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              assign [$$74] <- [$$p1.getField("entities").getField("urls")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ASSIGN  |PARTITIONED|
-                                project ([$$p1]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  select (neq($$71, "0")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- STREAM_SELECT  |PARTITIONED|
-                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      data-scan []<-[$$71, $$p1] <- test.ColumnDataset project ({entities:{urls:[{display_url:any,indices:any}]}}) filter on: eq(lowercase(scan-collection($$p1.getField("entities").getField("urls")).getField("display_url")), "string") [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|
+                        select (neq($$71, "0")) project: [$$p1] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- STREAM_SELECT  |PARTITIONED|
+                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            data-scan []<-[$$71, $$p1] <- test.ColumnDataset project ({entities:{urls:[{display_url:any,indices:any}]}}) filter on: eq(lowercase(scan-collection($$p1.getField("entities").getField("urls")).getField("display_url")), "string") [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/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 5bd0133..7ae6fe3 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
@@ -2,33 +2,27 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$22]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$22] <- [{"display_url": $$27}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$27]) [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 [$$25(ASC) ]  |PARTITIONED|
-            order (ASC, $$25) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$27, $$25]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$27] <- [get-item($$24, 0).getField("display_url")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$22] <- [{"display_url": $$27}] project: [$$22] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$27]) [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 [$$25(ASC) ]  |PARTITIONED|
+          order (ASC, $$25) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$27] <- [get-item($$24, 0).getField("display_url")] project: [$$27, $$25] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ASSIGN  |PARTITIONED|
+                select (eq(sql-count($$24), 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STREAM_SELECT  |PARTITIONED|
+                  assign [$$24] <- [$$p.getField("entities").getField("urls")] project: [$$25, $$24] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- ASSIGN  |PARTITIONED|
-                    select (eq(sql-count($$24), 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_SELECT  |PARTITIONED|
-                      project ([$$25, $$24]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$24] <- [$$p.getField("entities").getField("urls")] [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]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            data-scan []<-[$$25, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [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  |PARTITIONED|
+                      data-scan []<-[$$25, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [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/column/pushdown/other-pushdowns/other-pushdowns.014.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.014.plan
index f87b52f..d524b65 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.014.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.014.plan
@@ -2,110 +2,94 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$101]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$101] <- [{"uname": $$uname, "cnt": $$105}] [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 [$$105(DESC), $$uname(ASC) ]  |PARTITIONED|
-          order (DESC, $$105) (ASC, $$uname) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$105(DESC), $$uname(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              group by ([$$uname := $$116]) decor ([]) {
-                        aggregate [$$105] <- [agg-sql-sum($$115)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$116]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- HASH_PARTITION_EXCHANGE [$$116]  |PARTITIONED|
-                  group by ([$$116 := $$102]) decor ([]) {
-                            aggregate [$$115] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- AGGREGATE  |LOCAL|
-                              nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                         } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- SORT_GROUP_BY[$$102]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      project ([$$102]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$101] <- [{"uname": $$uname, "cnt": $$105}] project: [$$101] [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 [$$105(DESC), $$uname(ASC) ]  |PARTITIONED|
+        order (DESC, $$105) (ASC, $$uname) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$105(DESC), $$uname(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            group by ([$$uname := $$116]) decor ([]) {
+                      aggregate [$$105] <- [agg-sql-sum($$115)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$116]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- HASH_PARTITION_EXCHANGE [$$116]  |PARTITIONED|
+                group by ([$$116 := $$102]) decor ([]) {
+                          aggregate [$$115] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- AGGREGATE  |LOCAL|
+                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- SORT_GROUP_BY[$$102]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    select ($$92) project: [$$102] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- STREAM_SELECT  |PARTITIONED|
+                      project ([$$92, $$102]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- STREAM_PROJECT  |PARTITIONED|
-                        select ($$92) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_SELECT  |PARTITIONED|
-                          project ([$$92, $$102]) [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|
+                          group by ([$$114 := $$112]) decor ([$$102]) {
+                                    aggregate [$$92] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- AGGREGATE  |LOCAL|
+                                      select (not(is-missing($$113))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- STREAM_SELECT  |LOCAL|
+                                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- PRE_CLUSTERED_GROUP_BY[$$112]  |PARTITIONED|
                             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              group by ([$$114 := $$112]) decor ([$$102]) {
-                                        aggregate [$$92] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- AGGREGATE  |LOCAL|
-                                          select (not(is-missing($$113))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- STREAM_SELECT  |LOCAL|
-                                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- PRE_CLUSTERED_GROUP_BY[$$112]  |PARTITIONED|
+                              order (ASC, $$112) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- STABLE_SORT [$$112(ASC)]  |PARTITIONED|
                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  order (ASC, $$112) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- STABLE_SORT [$$112(ASC)]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$112]  |PARTITIONED|
+                                  project ([$$102, $$113, $$112]) [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]
-                                    -- HASH_PARTITION_EXCHANGE [$$112]  |PARTITIONED|
-                                      project ([$$102, $$113, $$112]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      left outer join (eq($$106, $$88)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- HYBRID_HASH_JOIN [$$106][$$88]  |PARTITIONED|
                                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          left outer join (eq($$106, $$88)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- HYBRID_HASH_JOIN [$$106][$$88]  |PARTITIONED|
-                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- HASH_PARTITION_EXCHANGE [$$106]  |PARTITIONED|
-                                              running-aggregate [$$112] <- [create-query-uid()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                project ([$$102, $$106]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  assign [$$106] <- [$$ht1.getField("display_url")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    project ([$$102, $$ht1]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      unnest $$ht1 <- scan-collection($$107) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- UNNEST  |PARTITIONED|
-                                                        project ([$$107, $$102]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          assign [$$107, $$102] <- [$$p1.getField("entities").getField("urls"), $$p1.getField("user").getField("name")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            project ([$$p1]) [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 []<-[$$103, $$p1] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]},user:{name:any}}) [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 [$$88]  |PARTITIONED|
-                                              project ([$$113, $$88]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                assign [$$113, $$88] <- [true, $$ht2.getField("display_url")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- HASH_PARTITION_EXCHANGE [$$106]  |PARTITIONED|
+                                          running-aggregate [$$112] <- [create-query-uid()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- RUNNING_AGGREGATE  |PARTITIONED|
+                                            assign [$$106] <- [$$ht1.getField("display_url")] project: [$$102, $$106] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                            -- ASSIGN  |PARTITIONED|
+                                              unnest $$ht1 <- scan-collection($$107) project: [$$102, $$ht1] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- UNNEST  |PARTITIONED|
+                                                assign [$$107, $$102] <- [$$p1.getField("entities").getField("urls"), $$p1.getField("user").getField("name")] project: [$$107, $$102] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                                 -- ASSIGN  |PARTITIONED|
-                                                  project ([$$ht2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                  project ([$$p1]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                                   -- STREAM_PROJECT  |PARTITIONED|
-                                                    unnest $$ht2 <- scan-collection($$108) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- UNNEST  |PARTITIONED|
-                                                      project ([$$108]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        assign [$$108] <- [$$p2.getField("entities").getField("urls")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          project ([$$p2]) [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 []<-[$$104, $$p2] <- test.RowDataset [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  |PARTITIONED|
+                                                      data-scan []<-[$$103, $$p1] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]},user:{name:any}}) [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 [$$88]  |PARTITIONED|
+                                          assign [$$113, $$88] <- [true, $$ht2.getField("display_url")] project: [$$113, $$88] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- ASSIGN  |PARTITIONED|
+                                            unnest $$ht2 <- scan-collection($$108) project: [$$ht2] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                            -- UNNEST  |PARTITIONED|
+                                              assign [$$108] <- [$$p2.getField("entities").getField("urls")] project: [$$108] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- ASSIGN  |PARTITIONED|
+                                                project ([$$p2]) [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 []<-[$$104, $$p2] <- test.RowDataset [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/column/pushdown/other-pushdowns/other-pushdowns.015.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.015.plan
index a2f87a4..f222777 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.015.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.015.plan
@@ -2,53 +2,47 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$52] <- [{"$1": $$57}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$57] <- [agg-sql-sum($$60)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$60] <- [agg-sql-count($$50)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              project ([$$50]) [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|
-                  join (eq($$55, $$56)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HYBRID_HASH_JOIN [$$55][$$56]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HASH_PARTITION_EXCHANGE [$$55]  |PARTITIONED|
-                      assign [$$50] <- [$$55.getField("y")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$52] <- [{"$1": $$57}] project: [$$52] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$57] <- [agg-sql-sum($$60)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$60] <- [agg-sql-count($$50)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            project ([$$50]) [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|
+                join (eq($$55, $$56)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HYBRID_HASH_JOIN [$$55][$$56]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HASH_PARTITION_EXCHANGE [$$55]  |PARTITIONED|
+                    assign [$$50] <- [$$55.getField("y")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ASSIGN  |PARTITIONED|
+                      assign [$$55] <- [$$c.getField("x")] project: [$$55] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- ASSIGN  |PARTITIONED|
-                        project ([$$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        project ([$$c]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$55] <- [$$c.getField("x")] [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|
+                            data-scan []<-[$$53, $$c] <- test.ColumnDataset project ({x:any}) [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|
-                                data-scan []<-[$$53, $$c] <- test.ColumnDataset project ({x:any}) [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 [$$56]  |PARTITIONED|
-                      project ([$$56]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                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 [$$56]  |PARTITIONED|
+                    assign [$$56] <- [$$r.getField("x")] project: [$$56] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ASSIGN  |PARTITIONED|
+                      project ([$$r]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$56] <- [$$r.getField("x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ASSIGN  |PARTITIONED|
-                          project ([$$r]) [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 []<-[$$54, $$r] <- test.RowDataset [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|
-                              data-scan []<-[$$54, $$r] <- test.RowDataset [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|
+                              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.016.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.016.plan
index fe8cca9..50a35be 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.016.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.016.plan
@@ -2,26 +2,37 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$65]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$65] <- [{"$1": $$67}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$67] <- [agg-sql-sum($$72)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$72] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              select (or($$53, $$57)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$53, $$57]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
+    assign [$$65] <- [{"$1": $$67}] project: [$$65] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$67] <- [agg-sql-sum($$72)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$72] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            select (or($$53, $$57)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$53, $$57]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_PROJECT  |PARTITIONED|
+                subplan {
+                          aggregate [$$57] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- AGGREGATE  |LOCAL|
+                            select (eq($$71, 2)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- STREAM_SELECT  |LOCAL|
+                              assign [$$71] <- [$$e.getField("x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- ASSIGN  |LOCAL|
+                                unnest $$e <- scan-collection($$68) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- SUBPLAN  |PARTITIONED|
                   subplan {
-                            aggregate [$$57] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            aggregate [$$53] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- AGGREGATE  |LOCAL|
-                              select (eq($$71, 2)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              select (eq($$69, 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                               -- STREAM_SELECT  |LOCAL|
-                                assign [$$71] <- [$$e.getField("x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                assign [$$69] <- [$$e.getField("x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                 -- ASSIGN  |LOCAL|
                                   unnest $$e <- scan-collection($$68) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                   -- UNNEST  |LOCAL|
@@ -29,30 +40,15 @@
                                     -- NESTED_TUPLE_SOURCE  |LOCAL|
                          } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- SUBPLAN  |PARTITIONED|
-                    subplan {
-                              aggregate [$$53] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- AGGREGATE  |LOCAL|
-                                select (eq($$69, 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |LOCAL|
-                                  assign [$$69] <- [$$e.getField("x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ASSIGN  |LOCAL|
-                                    unnest $$e <- scan-collection($$68) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- SUBPLAN  |PARTITIONED|
-                      project ([$$68]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    assign [$$68] <- [$$c.getField("val1")] project: [$$68] [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|
-                        assign [$$68] <- [$$c.getField("val1")] [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|
+                          data-scan []<-[$$66, $$c] <- test.ColumnDataset project ({val1:[{x:any}]}) filter on: or(eq(scan-collection($$c.getField("val1")).getField("x"), 1), eq(scan-collection($$c.getField("val1")).getField("x"), 2)) range-filter on: or(eq(scan-collection($$c.getField("val1")).getField("x"), 1), eq(scan-collection($$c.getField("val1")).getField("x"), 2)) [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|
-                              data-scan []<-[$$66, $$c] <- test.ColumnDataset project ({val1:[{x:any}]}) filter on: or(eq(scan-collection($$c.getField("val1")).getField("x"), 1), eq(scan-collection($$c.getField("val1")).getField("x"), 2)) range-filter on: or(eq(scan-collection($$c.getField("val1")).getField("x"), 1), eq(scan-collection($$c.getField("val1")).getField("x"), 2)) [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|
+                              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.017.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.017.plan
index 3d9bcfe..4a1bdc7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.017.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.017.plan
@@ -2,26 +2,37 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$65]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$65] <- [{"$1": $$67}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$67] <- [agg-sql-sum($$72)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$72] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              select (or($$53, $$57)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$53, $$57]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
+    assign [$$65] <- [{"$1": $$67}] project: [$$65] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$67] <- [agg-sql-sum($$72)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$72] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            select (or($$53, $$57)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$53, $$57]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_PROJECT  |PARTITIONED|
+                subplan {
+                          aggregate [$$57] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- AGGREGATE  |LOCAL|
+                            select (eq($$71, 2)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- STREAM_SELECT  |LOCAL|
+                              assign [$$71] <- [$$e.getField("y")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- ASSIGN  |LOCAL|
+                                unnest $$e <- scan-collection($$68) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- SUBPLAN  |PARTITIONED|
                   subplan {
-                            aggregate [$$57] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            aggregate [$$53] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- AGGREGATE  |LOCAL|
-                              select (eq($$71, 2)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              select (eq($$69, 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                               -- STREAM_SELECT  |LOCAL|
-                                assign [$$71] <- [$$e.getField("y")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                assign [$$69] <- [$$e.getField("x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                 -- ASSIGN  |LOCAL|
                                   unnest $$e <- scan-collection($$68) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                   -- UNNEST  |LOCAL|
@@ -29,30 +40,15 @@
                                     -- NESTED_TUPLE_SOURCE  |LOCAL|
                          } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- SUBPLAN  |PARTITIONED|
-                    subplan {
-                              aggregate [$$53] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- AGGREGATE  |LOCAL|
-                                select (eq($$69, 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |LOCAL|
-                                  assign [$$69] <- [$$e.getField("x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ASSIGN  |LOCAL|
-                                    unnest $$e <- scan-collection($$68) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- SUBPLAN  |PARTITIONED|
-                      project ([$$68]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    assign [$$68] <- [$$c.getField("val1")] project: [$$68] [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|
-                        assign [$$68] <- [$$c.getField("val1")] [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|
+                          data-scan []<-[$$66, $$c] <- test.ColumnDataset project ({val1:[{x:any,y:any}]}) filter on: or(eq(scan-collection($$c.getField("val1")).getField("x"), 1), eq(scan-collection($$c.getField("val1")).getField("y"), 2)) range-filter on: or(eq(scan-collection($$c.getField("val1")).getField("x"), 1), eq(scan-collection($$c.getField("val1")).getField("y"), 2)) [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|
-                              data-scan []<-[$$66, $$c] <- test.ColumnDataset project ({val1:[{x:any,y:any}]}) filter on: or(eq(scan-collection($$c.getField("val1")).getField("x"), 1), eq(scan-collection($$c.getField("val1")).getField("y"), 2)) range-filter on: or(eq(scan-collection($$c.getField("val1")).getField("x"), 1), eq(scan-collection($$c.getField("val1")).getField("y"), 2)) [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|
+                              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.018.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.018.plan
index 5e6a214..bf4a1ce 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.018.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.018.plan
@@ -2,59 +2,55 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$65]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$65] <- [{"$1": $$67}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$67] <- [agg-sql-sum($$72)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$72] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              select (or($$53, $$57)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$53, $$57]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  subplan {
-                            aggregate [$$57] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- AGGREGATE  |LOCAL|
-                              select (eq($$71, 2)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_SELECT  |LOCAL|
-                                assign [$$71] <- [$$e.getField("y")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- ASSIGN  |LOCAL|
-                                  unnest $$e <- scan-collection($$70) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- SUBPLAN  |PARTITIONED|
-                    project ([$$70, $$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      subplan {
-                                aggregate [$$53] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- AGGREGATE  |LOCAL|
-                                  select (eq($$69, 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- STREAM_SELECT  |LOCAL|
-                                    assign [$$69] <- [$$e.getField("x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- ASSIGN  |LOCAL|
-                                      unnest $$e <- scan-collection($$68) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- SUBPLAN  |PARTITIONED|
-                        project ([$$70, $$68]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$65] <- [{"$1": $$67}] project: [$$65] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$67] <- [agg-sql-sum($$72)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$72] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            select (or($$53, $$57)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$53, $$57]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_PROJECT  |PARTITIONED|
+                subplan {
+                          aggregate [$$57] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- AGGREGATE  |LOCAL|
+                            select (eq($$71, 2)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- STREAM_SELECT  |LOCAL|
+                              assign [$$71] <- [$$e.getField("y")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- ASSIGN  |LOCAL|
+                                unnest $$e <- scan-collection($$70) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- SUBPLAN  |PARTITIONED|
+                  project ([$$70, $$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    subplan {
+                              aggregate [$$53] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- AGGREGATE  |LOCAL|
+                                select (eq($$69, 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- STREAM_SELECT  |LOCAL|
+                                  assign [$$69] <- [$$e.getField("x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- ASSIGN  |LOCAL|
+                                    unnest $$e <- scan-collection($$68) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- SUBPLAN  |PARTITIONED|
+                      assign [$$70, $$68] <- [$$c.getField("val2"), $$c.getField("val1")] project: [$$70, $$68] [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|
-                          assign [$$70, $$68] <- [$$c.getField("val2"), $$c.getField("val1")] [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|
+                            data-scan []<-[$$66, $$c] <- test.ColumnDataset project ({val2:[{y:any}],val1:[{x:any}]}) filter on: or(eq(scan-collection($$c.getField("val1")).getField("x"), 1), eq(scan-collection($$c.getField("val2")).getField("y"), 2)) range-filter on: or(eq(scan-collection($$c.getField("val1")).getField("x"), 1), eq(scan-collection($$c.getField("val2")).getField("y"), 2)) [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|
-                                data-scan []<-[$$66, $$c] <- test.ColumnDataset project ({val2:[{y:any}],val1:[{x:any}]}) filter on: or(eq(scan-collection($$c.getField("val1")).getField("x"), 1), eq(scan-collection($$c.getField("val2")).getField("y"), 2)) range-filter on: or(eq(scan-collection($$c.getField("val1")).getField("x"), 1), eq(scan-collection($$c.getField("val2")).getField("y"), 2)) [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|
+                                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.019.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.019.plan
index 9089f9f..510b86c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.019.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.019.plan
@@ -2,23 +2,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$31]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$31] <- [{"$1": get-item(get-item(get-item($$32, 0).getField("f2"), 0), 0), "$2": get-item($$47.getField("f3"), 1), "$3": get-item(get-item($$47.getField("f2"), 1), 1)}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$31] <- [{"$1": get-item(get-item(get-item($$32, 0).getField("f2"), 0), 0), "$2": get-item($$47.getField("f3"), 1), "$3": get-item(get-item($$47.getField("f2"), 1), 1)}] project: [$$31] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      assign [$$47] <- [get-item($$32, 1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- ASSIGN  |PARTITIONED|
-        assign [$$47] <- [get-item($$32, 1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        assign [$$32] <- [$$c.getField("f1")] project: [$$32] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
         -- ASSIGN  |PARTITIONED|
-          project ([$$32]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          project ([$$c]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- STREAM_PROJECT  |PARTITIONED|
-            assign [$$32] <- [$$c.getField("f1")] [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|
+              data-scan []<-[$$33, $$c] <- test.ColumnDataset project ({f1:[{f2:[[any]],f3:[any]}]}) [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|
-                  data-scan []<-[$$33, $$c] <- test.ColumnDataset project ({f1:[{f2:[[any]],f3:[any]}]}) [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|
+                  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.020.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.020.plan
index b06cd7b..0c66acf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.020.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.020.plan
@@ -2,50 +2,44 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$66]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$66] <- [{"count": $$69}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$69] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_MERGE_EXCHANGE [$$64(ASC) ]  |PARTITIONED|
-            distinct ([$$64]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (ASC, $$64) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STABLE_SORT [$$64(ASC)]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HASH_PARTITION_EXCHANGE [$$64]  |PARTITIONED|
-                    project ([$$64]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$66] <- [{"count": $$69}] project: [$$66] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$69] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_MERGE_EXCHANGE [$$64(ASC) ]  |PARTITIONED|
+          distinct ([$$64]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              order (ASC, $$64) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STABLE_SORT [$$64(ASC)]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HASH_PARTITION_EXCHANGE [$$64]  |PARTITIONED|
+                  select (ge($$68, 3)) project: [$$64] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_SELECT  |PARTITIONED|
+                    project ([$$64, $$68]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                     -- STREAM_PROJECT  |PARTITIONED|
-                      select (ge($$68, 3)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_SELECT  |PARTITIONED|
-                        project ([$$64, $$68]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          subplan {
-                                    aggregate [$$68] <- [agg-sql-sum($$52)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- AGGREGATE  |LOCAL|
-                                      assign [$$52] <- [$$i.getField("count")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- ASSIGN  |LOCAL|
-                                        unnest $$i <- scan-collection($$70) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- SUBPLAN  |PARTITIONED|
-                            project ([$$64, $$70]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              assign [$$64, $$70] <- [to-object-var-str($$t).getField(0), $$t.getField("a1")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ASSIGN  |PARTITIONED|
-                                project ([$$t]) [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 []<-[$$67, $$t] <- test.ColumnDataset [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|
+                      subplan {
+                                aggregate [$$68] <- [agg-sql-sum($$52)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- AGGREGATE  |LOCAL|
+                                  assign [$$52] <- [$$i.getField("count")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- ASSIGN  |LOCAL|
+                                    unnest $$i <- scan-collection($$70) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- SUBPLAN  |PARTITIONED|
+                        assign [$$64, $$70] <- [to-object-var-str($$t).getField(0), $$t.getField("a1")] project: [$$64, $$70] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- ASSIGN  |PARTITIONED|
+                          project ([$$t]) [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 []<-[$$67, $$t] <- test.ColumnDataset [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/column/pushdown/other-pushdowns/other-pushdowns.021.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.021.plan
index b06cd7b..0c66acf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.021.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.021.plan
@@ -2,50 +2,44 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$66]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$66] <- [{"count": $$69}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$69] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_MERGE_EXCHANGE [$$64(ASC) ]  |PARTITIONED|
-            distinct ([$$64]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (ASC, $$64) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STABLE_SORT [$$64(ASC)]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HASH_PARTITION_EXCHANGE [$$64]  |PARTITIONED|
-                    project ([$$64]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$66] <- [{"count": $$69}] project: [$$66] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$69] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_MERGE_EXCHANGE [$$64(ASC) ]  |PARTITIONED|
+          distinct ([$$64]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              order (ASC, $$64) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STABLE_SORT [$$64(ASC)]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HASH_PARTITION_EXCHANGE [$$64]  |PARTITIONED|
+                  select (ge($$68, 3)) project: [$$64] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_SELECT  |PARTITIONED|
+                    project ([$$64, $$68]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                     -- STREAM_PROJECT  |PARTITIONED|
-                      select (ge($$68, 3)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_SELECT  |PARTITIONED|
-                        project ([$$64, $$68]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          subplan {
-                                    aggregate [$$68] <- [agg-sql-sum($$52)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- AGGREGATE  |LOCAL|
-                                      assign [$$52] <- [$$i.getField("count")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- ASSIGN  |LOCAL|
-                                        unnest $$i <- scan-collection($$70) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- SUBPLAN  |PARTITIONED|
-                            project ([$$64, $$70]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              assign [$$64, $$70] <- [to-object-var-str($$t).getField(0), $$t.getField("a1")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ASSIGN  |PARTITIONED|
-                                project ([$$t]) [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 []<-[$$67, $$t] <- test.ColumnDataset [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|
+                      subplan {
+                                aggregate [$$68] <- [agg-sql-sum($$52)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- AGGREGATE  |LOCAL|
+                                  assign [$$52] <- [$$i.getField("count")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- ASSIGN  |LOCAL|
+                                    unnest $$i <- scan-collection($$70) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- SUBPLAN  |PARTITIONED|
+                        assign [$$64, $$70] <- [to-object-var-str($$t).getField(0), $$t.getField("a1")] project: [$$64, $$70] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- ASSIGN  |PARTITIONED|
+                          project ([$$t]) [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 []<-[$$67, $$t] <- test.ColumnDataset [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/column/pushdown/other-pushdowns/other-pushdowns.022.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.022.plan
index 05bf25c..dfef008 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.022.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.022.plan
@@ -4,65 +4,59 @@
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit 5 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$84]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$84] <- [{"name": $$93, "phone": $$94, "num_reviews": sql-count($$91), "overall_avg": $#2, "overall_reviews": $$83}] [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 [$#2(DESC) ]  |PARTITIONED|
-            limit 5 [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]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (topK: 5) (DESC, $#2) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STABLE_SORT [topK: 5] [$#2(DESC)]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    subplan {
-                              aggregate [$$83] <- [listify($$82)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- AGGREGATE  |LOCAL|
-                                assign [$$82] <- [$$98.getField("Overall")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      assign [$$84] <- [{"name": $$93, "phone": $$94, "num_reviews": sql-count($$91), "overall_avg": $#2, "overall_reviews": $$83}] project: [$$84] [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 [$#2(DESC) ]  |PARTITIONED|
+          limit 5 [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]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              order (topK: 5) (DESC, $#2) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STABLE_SORT [topK: 5] [$#2(DESC)]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  subplan {
+                            aggregate [$$83] <- [listify($$82)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- AGGREGATE  |LOCAL|
+                              assign [$$82] <- [$$98.getField("Overall")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- ASSIGN  |LOCAL|
+                                assign [$$98] <- [$$reviews.getField("ratings")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                 -- ASSIGN  |LOCAL|
-                                  assign [$$98] <- [$$reviews.getField("ratings")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ASSIGN  |LOCAL|
-                                    unnest $$reviews <- scan-collection($$91) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- SUBPLAN  |PARTITIONED|
-                      project ([$$93, $$94, $$91, $#2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$#2] <- [get-item($$72, 0)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ASSIGN  |PARTITIONED|
-                          subplan {
-                                    aggregate [$$72] <- [listify($$86)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- AGGREGATE  |LOCAL|
-                                      aggregate [$$86] <- [agg-sql-min($$69)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- AGGREGATE  |LOCAL|
-                                        assign [$$69] <- [$$97.getField("Overall")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- ASSIGN  |LOCAL|
-                                          assign [$$97] <- [$$89.getField("ratings")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- ASSIGN  |LOCAL|
-                                            unnest $$89 <- scan-collection($$91) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- SUBPLAN  |PARTITIONED|
-                            project ([$$93, $$94, $$91]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  unnest $$reviews <- scan-collection($$91) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- SUBPLAN  |PARTITIONED|
+                    assign [$#2] <- [get-item($$72, 0)] project: [$$93, $$94, $$91, $#2] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ASSIGN  |PARTITIONED|
+                      subplan {
+                                aggregate [$$72] <- [listify($$86)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- AGGREGATE  |LOCAL|
+                                  aggregate [$$86] <- [agg-sql-min($$69)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- AGGREGATE  |LOCAL|
+                                    assign [$$69] <- [$$97.getField("Overall")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- ASSIGN  |LOCAL|
+                                      assign [$$97] <- [$$89.getField("ratings")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- ASSIGN  |LOCAL|
+                                        unnest $$89 <- scan-collection($$91) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- SUBPLAN  |PARTITIONED|
+                        select (eq($$ht.getField("city"), "Los Angeles")) project: [$$93, $$94, $$91] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- STREAM_SELECT  |PARTITIONED|
+                          assign [$$94, $$91, $$93] <- [$$ht.getField("phone"), $$ht.getField("reviews"), $$ht.getField("name")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- ASSIGN  |PARTITIONED|
+                            project ([$$ht]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- STREAM_PROJECT  |PARTITIONED|
-                              select (eq($$ht.getField("city"), "Los Angeles")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_SELECT  |PARTITIONED|
-                                assign [$$94, $$91, $$93] <- [$$ht.getField("phone"), $$ht.getField("reviews"), $$ht.getField("name")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- ASSIGN  |PARTITIONED|
-                                  project ([$$ht]) [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 []<-[$$85, $$ht] <- test.ColumnDataset project ({reviews:[{ratings:{Overall:any}}],phone:any,city:any,name:any}) filter on: eq($$ht.getField("city"), "Los Angeles") range-filter on: eq($$ht.getField("city"), "Los Angeles") [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  |PARTITIONED|
+                                data-scan []<-[$$85, $$ht] <- test.ColumnDataset project ({reviews:[{ratings:{Overall:any}}],phone:any,city:any,name:any}) filter on: eq($$ht.getField("city"), "Los Angeles") range-filter on: eq($$ht.getField("city"), "Los Angeles") [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/column/pushdown/other-pushdowns/other-pushdowns.023.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.023.plan
index 7ddc1a9..4708d89 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.023.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.023.plan
@@ -6,23 +6,19 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        project ([$$19]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          assign [$$19] <- [{"payload": $$20.getField("payload")}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- ASSIGN  |PARTITIONED|
-            limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_LIMIT  |PARTITIONED|
-              project ([$$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        assign [$$19] <- [{"payload": $$20.getField("payload")}] project: [$$19] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- ASSIGN  |PARTITIONED|
+          limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STREAM_LIMIT  |PARTITIONED|
+            assign [$$20] <- [$$o.getField("v")] project: [$$20] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ASSIGN  |PARTITIONED|
+              project ([$$o]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
               -- STREAM_PROJECT  |PARTITIONED|
-                assign [$$20] <- [$$o.getField("v")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ASSIGN  |PARTITIONED|
-                  project ([$$o]) [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 []<-[$$21, $$o] <- test.ColumnDataset condition (eq($$o.getField("v").getField("type"), "WeMo")) limit 10 project ({v:{payload:any,type:any}}) filter on: eq($$o.getField("v").getField("type"), "WeMo") range-filter on: eq($$o.getField("v").getField("type"), "WeMo") [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|
-                      data-scan []<-[$$21, $$o] <- test.ColumnDataset condition (eq($$o.getField("v").getField("type"), "WeMo")) limit 10 project ({v:{payload:any,type:any}}) filter on: eq($$o.getField("v").getField("type"), "WeMo") range-filter on: eq($$o.getField("v").getField("type"), "WeMo") [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|
+                      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.024.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.024.plan
index cc4aea0..e02e06b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.024.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.024.plan
@@ -2,37 +2,29 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$31]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$31] <- [{"$1": get-item(get-item($$36, 0), 0), "$2": get-item($$37, 1), "$3": get-item(get-item($$38, 1), 1)}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$36, $$37, $$38]) [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 [$$35(ASC) ]  |PARTITIONED|
-            order (ASC, $$35) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$36, $$37, $$38, $$35]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$38, $$37] <- [$$49.getField("f2"), $$49.getField("f3")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$31] <- [{"$1": get-item(get-item($$36, 0), 0), "$2": get-item($$37, 1), "$3": get-item(get-item($$38, 1), 1)}] project: [$$31] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$36, $$37, $$38]) [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 [$$35(ASC) ]  |PARTITIONED|
+          order (ASC, $$35) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$38, $$37] <- [$$49.getField("f2"), $$49.getField("f3")] project: [$$36, $$37, $$38, $$35] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ASSIGN  |PARTITIONED|
+                assign [$$49, $$36] <- [get-item($$33, 1), get-item($$33, 0).getField("f2")] project: [$$35, $$49, $$36] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  assign [$$33, $$35] <- [$$c.getField("f1"), $$c.getField("x")] project: [$$33, $$35] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- ASSIGN  |PARTITIONED|
-                    project ([$$35, $$49, $$36]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    project ([$$c]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                     -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$49, $$36] <- [get-item($$33, 1), get-item($$33, 0).getField("f2")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
-                        project ([$$33, $$35]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$33, $$35] <- [$$c.getField("f1"), $$c.getField("x")] [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|
-                                data-scan []<-[$$34, $$c] <- test.ColumnDataset project ({x:any,f1:[{f2:[[any]],f3:[any]}]}) [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  |PARTITIONED|
+                        data-scan []<-[$$34, $$c] <- test.ColumnDataset project ({x:any,f1:[{f2:[[any]],f3:[any]}]}) [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/column/pushdown/other-pushdowns/other-pushdowns.025.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.025.plan
index 830e096..567805d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.025.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.025.plan
@@ -2,21 +2,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$32]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$32] <- [{"$1": get-item(get-item(get-item($$33, 0).getField("f2"), 0), 0), "$2": get-item($$49.getField("f3"), 1), "$3": get-item(get-item($$49.getField("f2"), 1), 1), "x": $$c.getField("x")}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$32] <- [{"$1": get-item(get-item(get-item($$33, 0).getField("f2"), 0), 0), "$2": get-item($$49.getField("f3"), 1), "$3": get-item(get-item($$49.getField("f2"), 1), 1), "x": $$c.getField("x")}] project: [$$32] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      assign [$$49] <- [get-item($$33, 1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- ASSIGN  |PARTITIONED|
-        assign [$$49] <- [get-item($$33, 1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        assign [$$33] <- [$$c.getField("f1")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
         -- ASSIGN  |PARTITIONED|
-          assign [$$33] <- [$$c.getField("f1")] [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|
-                data-scan []<-[$$34, $$c] <- test.ColumnDataset project ({x:any,f1:[{f2:[[any]],f3:[any]}]}) [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|
+          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|
+              data-scan []<-[$$34, $$c] <- test.ColumnDataset project ({x:any,f1:[{f2:[[any]],f3:[any]}]}) [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/column/pushdown/other-pushdowns/other-pushdowns.026.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.026.plan
index b65bd37..e5f387d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.026.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.026.plan
@@ -2,37 +2,29 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$32]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$32] <- [{"$1": get-item(get-item($$37, 0), 0), "$2": get-item($$38, 1), "$3": get-item(get-item($$39, 1), 1), "x": $$40}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$37, $$38, $$39, $$40]) [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 [$$36(ASC) ]  |PARTITIONED|
-            order (ASC, $$36) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$37, $$38, $$39, $$40, $$36]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$39, $$38] <- [$$51.getField("f2"), $$51.getField("f3")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$32] <- [{"$1": get-item(get-item($$37, 0), 0), "$2": get-item($$38, 1), "$3": get-item(get-item($$39, 1), 1), "x": $$40}] project: [$$32] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$37, $$38, $$39, $$40]) [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 [$$36(ASC) ]  |PARTITIONED|
+          order (ASC, $$36) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$39, $$38] <- [$$51.getField("f2"), $$51.getField("f3")] project: [$$37, $$38, $$39, $$40, $$36] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ASSIGN  |PARTITIONED|
+                assign [$$51, $$37] <- [get-item($$34, 1), get-item($$34, 0).getField("f2")] project: [$$36, $$40, $$51, $$37] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  assign [$$34, $$36, $$40] <- [$$c.getField("f1"), $$c.getField("y"), $$c.getField("x")] project: [$$34, $$36, $$40] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- ASSIGN  |PARTITIONED|
-                    project ([$$36, $$40, $$51, $$37]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    project ([$$c]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                     -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$51, $$37] <- [get-item($$34, 1), get-item($$34, 0).getField("f2")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
-                        project ([$$34, $$36, $$40]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$34, $$36, $$40] <- [$$c.getField("f1"), $$c.getField("y"), $$c.getField("x")] [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|
-                                data-scan []<-[$$35, $$c] <- test.ColumnDataset project ({x:any,y:any,f1:[{f2:[[any]],f3:[any]}]}) [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  |PARTITIONED|
+                        data-scan []<-[$$35, $$c] <- test.ColumnDataset project ({x:any,y:any,f1:[{f2:[[any]],f3:[any]}]}) [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/column/pushdown/other-pushdowns/other-pushdowns.027.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.027.plan
index 64d7359..e16744d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.027.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.027.plan
@@ -2,25 +2,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$35]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$35] <- [{"$1": get-item(get-item(get-item($$36, 0).getField("f2"), 0), 0), "$2": get-item($$43, 1), "$3": get-item(get-item($$55.getField("f2"), 1), 1), "f4": $$43.getField("f4")}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$35] <- [{"$1": get-item(get-item(get-item($$36, 0).getField("f2"), 0), 0), "$2": get-item($$43, 1), "$3": get-item(get-item($$55.getField("f2"), 1), 1), "f4": $$43.getField("f4")}] project: [$$35] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      assign [$$43] <- [$$55.getField("f3")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- ASSIGN  |PARTITIONED|
-        assign [$$43] <- [$$55.getField("f3")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        assign [$$55] <- [get-item($$36, 1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
         -- ASSIGN  |PARTITIONED|
-          assign [$$55] <- [get-item($$36, 1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          assign [$$36] <- [$$c.getField("f1")] project: [$$36] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- ASSIGN  |PARTITIONED|
-            project ([$$36]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            project ([$$c]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
             -- STREAM_PROJECT  |PARTITIONED|
-              assign [$$36] <- [$$c.getField("f1")] [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|
+                data-scan []<-[$$37, $$c] <- test.ColumnDataset project ({f1:[{f2:[[any]],f3:<[any],{f4:any}>}]}) [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|
-                    data-scan []<-[$$37, $$c] <- test.ColumnDataset project ({f1:[{f2:[[any]],f3:<[any],{f4:any}>}]}) [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|
+                    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/replicate/replicate.010.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.010.plan
index aa9bed7..ff20185 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.010.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.010.plan
@@ -2,281 +2,243 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$223]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$223] <- [{"su_suppkey": $$230, "su_name": $$272, "su_address": $$273, "su_phone": $$274, "total_revenue": $$231}] [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 [$$230(ASC) ]  |PARTITIONED|
-          order (ASC, $$230) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$230(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$230, $$272, $$273, $$274, $$231]) [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|
-                  join (eq($$230, $#1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HYBRID_HASH_JOIN [$#1][$$230]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
-                      project ([$$231, $#1]) [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|
-                          join (eq($$231, $$268)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- HYBRID_HASH_JOIN [$$231][$$268]  |PARTITIONED|
-                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- HASH_PARTITION_EXCHANGE [$$231]  |PARTITIONED|
-                              project ([$$231, $#1]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                assign [$$231] <- [{"supplier_no": $#1, "total_rev": $$240}.getField("total_revenue")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- ASSIGN  |PARTITIONED|
+    assign [$$223] <- [{"su_suppkey": $$230, "su_name": $$272, "su_address": $$273, "su_phone": $$274, "total_revenue": $$231}] project: [$$223] [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 [$$230(ASC) ]  |PARTITIONED|
+        order (ASC, $$230) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$230(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            project ([$$230, $$272, $$273, $$274, $$231]) [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|
+                join (eq($$230, $#1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HYBRID_HASH_JOIN [$#1][$$230]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                    project ([$$231, $#1]) [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|
+                        join (eq($$231, $$268)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- HYBRID_HASH_JOIN [$$231][$$268]  |PARTITIONED|
+                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- HASH_PARTITION_EXCHANGE [$$231]  |PARTITIONED|
+                            assign [$$231] <- [{"supplier_no": $#1, "total_rev": $$240}.getField("total_revenue")] project: [$$231, $#1] [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]
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                group by ([$#1 := $$280]) decor ([]) {
+                                          aggregate [$$240] <- [agg-global-sql-sum($$279)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- AGGREGATE  |LOCAL|
+                                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- SORT_GROUP_BY[$$280]  |PARTITIONED|
                                   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    group by ([$#1 := $$280]) decor ([]) {
-                                              aggregate [$$240] <- [agg-global-sql-sum($$279)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- HASH_PARTITION_EXCHANGE [$$280]  |PARTITIONED|
+                                    group by ([$$280 := $$225]) decor ([]) {
+                                              aggregate [$$279] <- [agg-local-sql-sum($$169)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                               -- AGGREGATE  |LOCAL|
                                                 nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                                            } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- SORT_GROUP_BY[$$280]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$225]  |PARTITIONED|
                                       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- HASH_PARTITION_EXCHANGE [$$280]  |PARTITIONED|
-                                        group by ([$$280 := $$225]) decor ([]) {
-                                                  aggregate [$$279] <- [agg-local-sql-sum($$169)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- AGGREGATE  |LOCAL|
-                                                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- SORT_GROUP_BY[$$225]  |PARTITIONED|
-                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            project ([$$169, $$225]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              assign [$$225] <- [numeric-mod(numeric-multiply($$228, $$229), 10000)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- ASSIGN  |PARTITIONED|
-                                                project ([$$228, $$229, $$169]) [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|
-                                                    join (and(eq($$262, $$229), eq($$263, $$228))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- HYBRID_HASH_JOIN [$$229, $$228][$$262, $$263]  |PARTITIONED|
-                                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        project ([$$228, $$229]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          assign [$$228, $$229] <- [$$250, $$251] [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]
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                              -- REPLICATE  |PARTITIONED|
-                                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                -- HASH_PARTITION_EXCHANGE [$$251, $$250]  |PARTITIONED|
-                                                                  project ([$$250, $$251]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    assign [$$251, $$250] <- [$$253.getField("s_i_id"), $$253.getField("s_w_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                    -- ASSIGN  |PARTITIONED|
-                                                                      project ([$$253]) [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 []<-[$$258, $$253] <- test.stock project ({s_w_id:any,s_i_id:any}) [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 [$$262, $$263]  |PARTITIONED|
-                                                        project ([$$169, $$262, $$263]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          select (and(ge($$227, "2018-01-01 00:00:00.000000"), lt($$227, "2018-04-01 00:00:00.000000"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                          -- STREAM_SELECT  |PARTITIONED|
-                                                            project ([$$169, $$263, $$262, $$227]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              assign [$$169, $$263, $$262, $$227] <- [$$259, $$267, $$266, $$256] [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]
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                  -- REPLICATE  |PARTITIONED|
-                                                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      project ([$$259, $$267, $$266, $$256]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        assign [$$259, $$267, $$266, $$256] <- [$$255.getField("ol_amount"), $$255.getField("ol_supply_w_id"), $$255.getField("ol_i_id"), $$255.getField("ol_delivery_d")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                        -- ASSIGN  |PARTITIONED|
-                                                                          project ([$$255]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            unnest $$255 <- scan-collection($$270) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                            -- UNNEST  |PARTITIONED|
-                                                                              project ([$$270]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                assign [$$270] <- [$$254.getField("o_orderline")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                  project ([$$254]) [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 [$$257, $$254] <- index-search("orders", 0, "Default", "test", "orders", false, false, 1, $$291, 1, $$291, true, true, true) project ({o_orderline:[{ol_delivery_d:any,ol_i_id:any,ol_supply_w_id:any,ol_amount:any}]}) range-filter on: and(ge(scan-collection($$254.getField("o_orderline")).getField("ol_delivery_d"), "2018-01-01 00:00:00.000000"), lt(scan-collection($$254.getField("o_orderline")).getField("ol_delivery_d"), "2018-04-01 00:00:00.000000")) [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|
-                                                                                          distinct ([$$291]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                                                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                              order (ASC, $$291) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                              -- STABLE_SORT [$$291(ASC)]  |PARTITIONED|
-                                                                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                  project ([$$291]) [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 [$$290, $$291] <- index-search("orderline_delivery_d", 0, "Default", "test", "orders", false, false, 1, $$288, 1, $$289, 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 [$$288, $$289] <- ["2018-01-01 00:00:00.000000", "2018-04-01 00:00:00.000000"] [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|
-                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- HASH_PARTITION_EXCHANGE [$$268]  |PARTITIONED|
-                              project ([$$268]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |UNPARTITIONED|
-                                assign [$$268] <- [get-item($$213, 0)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- ASSIGN  |UNPARTITIONED|
-                                  aggregate [$$213] <- [listify($$278)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- AGGREGATE  |UNPARTITIONED|
-                                    aggregate [$$278] <- [agg-global-sql-max($$281)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- AGGREGATE  |UNPARTITIONED|
-                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                        aggregate [$$281] <- [agg-local-sql-max($$210)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- AGGREGATE  |PARTITIONED|
-                                          project ([$$210]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        assign [$$225] <- [numeric-mod(numeric-multiply($$228, $$229), 10000)] project: [$$169, $$225] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- ASSIGN  |PARTITIONED|
+                                          project ([$$228, $$229, $$169]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                           -- STREAM_PROJECT  |PARTITIONED|
-                                            assign [$$210] <- [{"supplier_no": $$247, "total_rev": $$265}.getField("total_revenue")] [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]
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                group by ([$$247 := $$283]) decor ([]) {
-                                                          aggregate [$$265] <- [agg-global-sql-sum($$282)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                          -- AGGREGATE  |LOCAL|
-                                                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- SORT_GROUP_BY[$$283]  |PARTITIONED|
-                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- HASH_PARTITION_EXCHANGE [$$283]  |PARTITIONED|
-                                                    group by ([$$283 := $$249]) decor ([]) {
-                                                              aggregate [$$282] <- [agg-local-sql-sum($$259)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                              -- AGGREGATE  |LOCAL|
-                                                                nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                           } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- SORT_GROUP_BY[$$249]  |PARTITIONED|
-                                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        project ([$$259, $$249]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          assign [$$249] <- [numeric-mod(numeric-multiply($$250, $$251), 10000)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              join (and(eq($$262, $$229), eq($$263, $$228))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- HYBRID_HASH_JOIN [$$229, $$228][$$262, $$263]  |PARTITIONED|
+                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  assign [$$228, $$229] <- [$$250, $$251] project: [$$228, $$229] [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]
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                        -- HASH_PARTITION_EXCHANGE [$$251, $$250]  |PARTITIONED|
+                                                          assign [$$251, $$250] <- [$$253.getField("s_i_id"), $$253.getField("s_w_id")] project: [$$250, $$251] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                                           -- ASSIGN  |PARTITIONED|
-                                                            project ([$$250, $$251, $$259]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                            project ([$$253]) [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|
-                                                                join (and(eq($$266, $$251), eq($$267, $$250))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                -- HYBRID_HASH_JOIN [$$251, $$250][$$266, $$267]  |PARTITIONED|
+                                                                data-scan []<-[$$258, $$253] <- test.stock project ({s_w_id:any,s_i_id:any}) [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|
-                                                                    replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                    -- REPLICATE  |PARTITIONED|
-                                                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                      -- HASH_PARTITION_EXCHANGE [$$251, $$250]  |PARTITIONED|
-                                                                        project ([$$250, $$251]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          assign [$$251, $$250] <- [$$253.getField("s_i_id"), $$253.getField("s_w_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            project ([$$253]) [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 []<-[$$258, $$253] <- test.stock project ({s_w_id:any,s_i_id:any}) [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 [$$266, $$267]  |PARTITIONED|
-                                                                    project ([$$259, $$266, $$267]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                      select (and(lt($$256, "2018-04-01 00:00:00.000000"), ge($$256, "2018-01-01 00:00:00.000000"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                      -- STREAM_SELECT  |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 [$$262, $$263]  |PARTITIONED|
+                                                  select (and(ge($$227, "2018-01-01 00:00:00.000000"), lt($$227, "2018-04-01 00:00:00.000000"))) project: [$$169, $$262, $$263] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                    assign [$$169, $$263, $$262, $$227] <- [$$259, $$267, $$266, $$256] project: [$$169, $$263, $$262, $$227] [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]
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                        -- REPLICATE  |PARTITIONED|
+                                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            assign [$$259, $$267, $$266, $$256] <- [$$255.getField("ol_amount"), $$255.getField("ol_supply_w_id"), $$255.getField("ol_i_id"), $$255.getField("ol_delivery_d")] project: [$$259, $$267, $$266, $$256] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              unnest $$255 <- scan-collection($$270) project: [$$255] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                              -- UNNEST  |PARTITIONED|
+                                                                assign [$$270] <- [$$254.getField("o_orderline")] project: [$$270] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  project ([$$254]) [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 [$$257, $$254] <- index-search("orders", 0, "Default", "test", "orders", false, false, 1, $$291, 1, $$291, true, true, true) project ({o_orderline:[{ol_delivery_d:any,ol_i_id:any,ol_supply_w_id:any,ol_amount:any}]}) range-filter on: and(ge(scan-collection($$254.getField("o_orderline")).getField("ol_delivery_d"), "2018-01-01 00:00:00.000000"), lt(scan-collection($$254.getField("o_orderline")).getField("ol_delivery_d"), "2018-04-01 00:00:00.000000")) [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|
-                                                                          replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                          -- REPLICATE  |PARTITIONED|
+                                                                          distinct ([$$291]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
                                                                             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              project ([$$259, $$267, $$266, $$256]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                assign [$$259, $$267, $$266, $$256] <- [$$255.getField("ol_amount"), $$255.getField("ol_supply_w_id"), $$255.getField("ol_i_id"), $$255.getField("ol_delivery_d")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                  project ([$$255]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                              order (ASC, $$291) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                              -- STABLE_SORT [$$291(ASC)]  |PARTITIONED|
+                                                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  project ([$$291]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                                                                   -- STREAM_PROJECT  |PARTITIONED|
-                                                                                    unnest $$255 <- scan-collection($$270) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                    -- UNNEST  |PARTITIONED|
-                                                                                      project ([$$270]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                        assign [$$270] <- [$$254.getField("o_orderline")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                        -- ASSIGN  |PARTITIONED|
-                                                                                          project ([$$254]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      unnest-map [$$290, $$291] <- index-search("orderline_delivery_d", 0, "Default", "test", "orders", false, false, 1, $$288, 1, $$289, 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 [$$288, $$289] <- ["2018-01-01 00:00:00.000000", "2018-04-01 00:00:00.000000"] [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|
+                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- HASH_PARTITION_EXCHANGE [$$268]  |PARTITIONED|
+                            assign [$$268] <- [get-item($$213, 0)] project: [$$268] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- ASSIGN  |UNPARTITIONED|
+                              aggregate [$$213] <- [listify($$278)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- AGGREGATE  |UNPARTITIONED|
+                                aggregate [$$278] <- [agg-global-sql-max($$281)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- AGGREGATE  |UNPARTITIONED|
+                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                    aggregate [$$281] <- [agg-local-sql-max($$210)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- AGGREGATE  |PARTITIONED|
+                                      assign [$$210] <- [{"supplier_no": $$247, "total_rev": $$265}.getField("total_revenue")] project: [$$210] [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]
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          group by ([$$247 := $$283]) decor ([]) {
+                                                    aggregate [$$265] <- [agg-global-sql-sum($$282)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                    -- AGGREGATE  |LOCAL|
+                                                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- SORT_GROUP_BY[$$283]  |PARTITIONED|
+                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                            -- HASH_PARTITION_EXCHANGE [$$283]  |PARTITIONED|
+                                              group by ([$$283 := $$249]) decor ([]) {
+                                                        aggregate [$$282] <- [agg-local-sql-sum($$259)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                        -- AGGREGATE  |LOCAL|
+                                                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- SORT_GROUP_BY[$$249]  |PARTITIONED|
+                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  assign [$$249] <- [numeric-mod(numeric-multiply($$250, $$251), 10000)] project: [$$259, $$249] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    project ([$$250, $$251, $$259]) [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|
+                                                        join (and(eq($$266, $$251), eq($$267, $$250))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                        -- HYBRID_HASH_JOIN [$$251, $$250][$$266, $$267]  |PARTITIONED|
+                                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                            -- REPLICATE  |PARTITIONED|
+                                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                              -- HASH_PARTITION_EXCHANGE [$$251, $$250]  |PARTITIONED|
+                                                                assign [$$251, $$250] <- [$$253.getField("s_i_id"), $$253.getField("s_w_id")] project: [$$250, $$251] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  project ([$$253]) [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 []<-[$$258, $$253] <- test.stock project ({s_w_id:any,s_i_id:any}) [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 [$$266, $$267]  |PARTITIONED|
+                                                            select (and(lt($$256, "2018-04-01 00:00:00.000000"), ge($$256, "2018-01-01 00:00:00.000000"))) project: [$$259, $$266, $$267] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                -- REPLICATE  |PARTITIONED|
+                                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    assign [$$259, $$267, $$266, $$256] <- [$$255.getField("ol_amount"), $$255.getField("ol_supply_w_id"), $$255.getField("ol_i_id"), $$255.getField("ol_delivery_d")] project: [$$259, $$267, $$266, $$256] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      unnest $$255 <- scan-collection($$270) project: [$$255] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                      -- UNNEST  |PARTITIONED|
+                                                                        assign [$$270] <- [$$254.getField("o_orderline")] project: [$$270] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          project ([$$254]) [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 [$$257, $$254] <- index-search("orders", 0, "Default", "test", "orders", false, false, 1, $$291, 1, $$291, true, true, true) project ({o_orderline:[{ol_delivery_d:any,ol_i_id:any,ol_supply_w_id:any,ol_amount:any}]}) range-filter on: and(ge(scan-collection($$254.getField("o_orderline")).getField("ol_delivery_d"), "2018-01-01 00:00:00.000000"), lt(scan-collection($$254.getField("o_orderline")).getField("ol_delivery_d"), "2018-04-01 00:00:00.000000")) [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|
+                                                                                  distinct ([$$291]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                                                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      order (ASC, $$291) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                      -- STABLE_SORT [$$291(ASC)]  |PARTITIONED|
+                                                                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          project ([$$291]) [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 [$$257, $$254] <- index-search("orders", 0, "Default", "test", "orders", false, false, 1, $$291, 1, $$291, true, true, true) project ({o_orderline:[{ol_delivery_d:any,ol_i_id:any,ol_supply_w_id:any,ol_amount:any}]}) range-filter on: and(ge(scan-collection($$254.getField("o_orderline")).getField("ol_delivery_d"), "2018-01-01 00:00:00.000000"), lt(scan-collection($$254.getField("o_orderline")).getField("ol_delivery_d"), "2018-04-01 00:00:00.000000")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                              unnest-map [$$290, $$291] <- index-search("orderline_delivery_d", 0, "Default", "test", "orders", false, false, 1, $$288, 1, $$289, 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|
-                                                                                                  distinct ([$$291]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                  -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-                                                                                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                      order (ASC, $$291) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                      -- STABLE_SORT [$$291(ASC)]  |PARTITIONED|
-                                                                                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          project ([$$291]) [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 [$$290, $$291] <- index-search("orderline_delivery_d", 0, "Default", "test", "orders", false, false, 1, $$288, 1, $$289, 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 [$$288, $$289] <- ["2018-01-01 00:00:00.000000", "2018-04-01 00:00:00.000000"] [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|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HASH_PARTITION_EXCHANGE [$$230]  |PARTITIONED|
-                      project ([$$230, $$272, $$273, $$274]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                                  assign [$$288, $$289] <- ["2018-01-01 00:00:00.000000", "2018-04-01 00:00:00.000000"] [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|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HASH_PARTITION_EXCHANGE [$$230]  |PARTITIONED|
+                    assign [$$274, $$273, $$272, $$230] <- [$$su.getField("su_phone"), $$su.getField("su_address"), $$su.getField("su_name"), $$su.getField("su_suppkey")] project: [$$230, $$272, $$273, $$274] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ASSIGN  |PARTITIONED|
+                      project ([$$su]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$274, $$273, $$272, $$230] <- [$$su.getField("su_phone"), $$su.getField("su_address"), $$su.getField("su_name"), $$su.getField("su_suppkey")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ASSIGN  |PARTITIONED|
-                          project ([$$su]) [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 []<-[$$234, $$su] <- test.supplier project ({su_phone:any,su_suppkey:any,su_name:any,su_address:any}) [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|
-                              data-scan []<-[$$234, $$su] <- test.supplier project ({su_phone:any,su_suppkey:any,su_name:any,su_address:any}) [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|
+                              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/replicate/replicate.011.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.011.plan
index 5abc9f5..6be8f9a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.011.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.011.plan
@@ -2,245 +2,207 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$223]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$223] <- [{"su_suppkey": $$230, "su_name": $$272, "su_address": $$273, "su_phone": $$274, "total_revenue": $$231}] [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 [$$230(ASC) ]  |PARTITIONED|
-          order (ASC, $$230) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$230(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$230, $$272, $$273, $$274, $$231]) [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|
-                  join (eq($$230, $#1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HYBRID_HASH_JOIN [$#1][$$230]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
-                      project ([$$231, $#1]) [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|
-                          join (eq($$231, $$268)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- HYBRID_HASH_JOIN [$$231][$$268]  |PARTITIONED|
-                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- HASH_PARTITION_EXCHANGE [$$231]  |PARTITIONED|
-                              project ([$$231, $#1]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                assign [$$231] <- [{"supplier_no": $#1, "total_rev": $$240}.getField("total_revenue")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- ASSIGN  |PARTITIONED|
+    assign [$$223] <- [{"su_suppkey": $$230, "su_name": $$272, "su_address": $$273, "su_phone": $$274, "total_revenue": $$231}] project: [$$223] [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 [$$230(ASC) ]  |PARTITIONED|
+        order (ASC, $$230) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$230(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            project ([$$230, $$272, $$273, $$274, $$231]) [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|
+                join (eq($$230, $#1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HYBRID_HASH_JOIN [$#1][$$230]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                    project ([$$231, $#1]) [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|
+                        join (eq($$231, $$268)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- HYBRID_HASH_JOIN [$$231][$$268]  |PARTITIONED|
+                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- HASH_PARTITION_EXCHANGE [$$231]  |PARTITIONED|
+                            assign [$$231] <- [{"supplier_no": $#1, "total_rev": $$240}.getField("total_revenue")] project: [$$231, $#1] [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]
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                group by ([$#1 := $$280]) decor ([]) {
+                                          aggregate [$$240] <- [agg-global-sql-sum($$279)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- AGGREGATE  |LOCAL|
+                                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- SORT_GROUP_BY[$$280]  |PARTITIONED|
                                   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    group by ([$#1 := $$280]) decor ([]) {
-                                              aggregate [$$240] <- [agg-global-sql-sum($$279)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- HASH_PARTITION_EXCHANGE [$$280]  |PARTITIONED|
+                                    group by ([$$280 := $$225]) decor ([]) {
+                                              aggregate [$$279] <- [agg-local-sql-sum($$169)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                               -- AGGREGATE  |LOCAL|
                                                 nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                                            } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- SORT_GROUP_BY[$$280]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$225]  |PARTITIONED|
                                       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- HASH_PARTITION_EXCHANGE [$$280]  |PARTITIONED|
-                                        group by ([$$280 := $$225]) decor ([]) {
-                                                  aggregate [$$279] <- [agg-local-sql-sum($$169)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- AGGREGATE  |LOCAL|
-                                                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- SORT_GROUP_BY[$$225]  |PARTITIONED|
-                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            project ([$$169, $$225]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              assign [$$225] <- [numeric-mod(numeric-multiply($$228, $$229), 10000)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- ASSIGN  |PARTITIONED|
-                                                project ([$$228, $$229, $$169]) [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|
-                                                    join (and(eq($$262, $$229), eq($$263, $$228))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- HYBRID_HASH_JOIN [$$229, $$228][$$262, $$263]  |PARTITIONED|
-                                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        project ([$$228, $$229]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          assign [$$228, $$229] <- [$$250, $$251] [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]
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                              -- REPLICATE  |PARTITIONED|
-                                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                -- HASH_PARTITION_EXCHANGE [$$251, $$250]  |PARTITIONED|
-                                                                  project ([$$250, $$251]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    assign [$$251, $$250] <- [$$253.getField("s_i_id"), $$253.getField("s_w_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                    -- ASSIGN  |PARTITIONED|
-                                                                      project ([$$253]) [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 []<-[$$258, $$253] <- test.stock project ({s_w_id:any,s_i_id:any}) [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 [$$262, $$263]  |PARTITIONED|
-                                                        project ([$$169, $$262, $$263]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          select (and(ge($$227, "2018-01-01 00:00:00.000000"), lt($$227, "2018-04-01 00:00:00.000000"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                          -- STREAM_SELECT  |PARTITIONED|
-                                                            project ([$$169, $$263, $$262, $$227]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              assign [$$169, $$263, $$262, $$227] <- [$$259, $$267, $$266, $$256] [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]
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                  -- REPLICATE  |PARTITIONED|
-                                                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      project ([$$259, $$267, $$266, $$256]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        assign [$$259, $$267, $$266, $$256] <- [$$255.getField("ol_amount"), $$255.getField("ol_supply_w_id"), $$255.getField("ol_i_id"), $$255.getField("ol_delivery_d")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                        -- ASSIGN  |PARTITIONED|
-                                                                          project ([$$255]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            unnest $$255 <- scan-collection($$270) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                            -- UNNEST  |PARTITIONED|
-                                                                              project ([$$270]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                assign [$$270] <- [$$254.getField("o_orderline")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                  project ([$$254]) [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 []<-[$$257, $$254] <- test.orders project ({o_orderline:[{ol_delivery_d:any,ol_i_id:any,ol_supply_w_id:any,ol_amount:any}]}) filter on: and(ge(scan-collection($$254.getField("o_orderline")).getField("ol_delivery_d"), "2018-01-01 00:00:00.000000"), lt(scan-collection($$254.getField("o_orderline")).getField("ol_delivery_d"), "2018-04-01 00:00:00.000000")) range-filter on: and(ge(scan-collection($$254.getField("o_orderline")).getField("ol_delivery_d"), "2018-01-01 00:00:00.000000"), lt(scan-collection($$254.getField("o_orderline")).getField("ol_delivery_d"), "2018-04-01 00:00:00.000000")) [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 [$$268]  |PARTITIONED|
-                              project ([$$268]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |UNPARTITIONED|
-                                assign [$$268] <- [get-item($$213, 0)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- ASSIGN  |UNPARTITIONED|
-                                  aggregate [$$213] <- [listify($$278)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- AGGREGATE  |UNPARTITIONED|
-                                    aggregate [$$278] <- [agg-global-sql-max($$281)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- AGGREGATE  |UNPARTITIONED|
-                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                        aggregate [$$281] <- [agg-local-sql-max($$210)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- AGGREGATE  |PARTITIONED|
-                                          project ([$$210]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        assign [$$225] <- [numeric-mod(numeric-multiply($$228, $$229), 10000)] project: [$$169, $$225] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- ASSIGN  |PARTITIONED|
+                                          project ([$$228, $$229, $$169]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                           -- STREAM_PROJECT  |PARTITIONED|
-                                            assign [$$210] <- [{"supplier_no": $$247, "total_rev": $$265}.getField("total_revenue")] [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]
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                group by ([$$247 := $$283]) decor ([]) {
-                                                          aggregate [$$265] <- [agg-global-sql-sum($$282)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                          -- AGGREGATE  |LOCAL|
-                                                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- SORT_GROUP_BY[$$283]  |PARTITIONED|
-                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- HASH_PARTITION_EXCHANGE [$$283]  |PARTITIONED|
-                                                    group by ([$$283 := $$249]) decor ([]) {
-                                                              aggregate [$$282] <- [agg-local-sql-sum($$259)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                              -- AGGREGATE  |LOCAL|
-                                                                nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                           } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- SORT_GROUP_BY[$$249]  |PARTITIONED|
-                                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        project ([$$259, $$249]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          assign [$$249] <- [numeric-mod(numeric-multiply($$250, $$251), 10000)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              join (and(eq($$262, $$229), eq($$263, $$228))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- HYBRID_HASH_JOIN [$$229, $$228][$$262, $$263]  |PARTITIONED|
+                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  assign [$$228, $$229] <- [$$250, $$251] project: [$$228, $$229] [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]
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                        -- HASH_PARTITION_EXCHANGE [$$251, $$250]  |PARTITIONED|
+                                                          assign [$$251, $$250] <- [$$253.getField("s_i_id"), $$253.getField("s_w_id")] project: [$$250, $$251] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                                           -- ASSIGN  |PARTITIONED|
-                                                            project ([$$250, $$251, $$259]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                            project ([$$253]) [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|
-                                                                join (and(eq($$266, $$251), eq($$267, $$250))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                -- HYBRID_HASH_JOIN [$$251, $$250][$$266, $$267]  |PARTITIONED|
+                                                                data-scan []<-[$$258, $$253] <- test.stock project ({s_w_id:any,s_i_id:any}) [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|
-                                                                    replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                    -- REPLICATE  |PARTITIONED|
-                                                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                      -- HASH_PARTITION_EXCHANGE [$$251, $$250]  |PARTITIONED|
-                                                                        project ([$$250, $$251]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          assign [$$251, $$250] <- [$$253.getField("s_i_id"), $$253.getField("s_w_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            project ([$$253]) [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 []<-[$$258, $$253] <- test.stock project ({s_w_id:any,s_i_id:any}) [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 [$$266, $$267]  |PARTITIONED|
-                                                                    project ([$$259, $$266, $$267]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                      select (and(lt($$256, "2018-04-01 00:00:00.000000"), ge($$256, "2018-01-01 00:00:00.000000"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                      -- STREAM_SELECT  |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 [$$262, $$263]  |PARTITIONED|
+                                                  select (and(ge($$227, "2018-01-01 00:00:00.000000"), lt($$227, "2018-04-01 00:00:00.000000"))) project: [$$169, $$262, $$263] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                    assign [$$169, $$263, $$262, $$227] <- [$$259, $$267, $$266, $$256] project: [$$169, $$263, $$262, $$227] [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]
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                        -- REPLICATE  |PARTITIONED|
+                                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            assign [$$259, $$267, $$266, $$256] <- [$$255.getField("ol_amount"), $$255.getField("ol_supply_w_id"), $$255.getField("ol_i_id"), $$255.getField("ol_delivery_d")] project: [$$259, $$267, $$266, $$256] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              unnest $$255 <- scan-collection($$270) project: [$$255] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                              -- UNNEST  |PARTITIONED|
+                                                                assign [$$270] <- [$$254.getField("o_orderline")] project: [$$270] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  project ([$$254]) [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 []<-[$$257, $$254] <- test.orders project ({o_orderline:[{ol_delivery_d:any,ol_i_id:any,ol_supply_w_id:any,ol_amount:any}]}) filter on: and(ge(scan-collection($$254.getField("o_orderline")).getField("ol_delivery_d"), "2018-01-01 00:00:00.000000"), lt(scan-collection($$254.getField("o_orderline")).getField("ol_delivery_d"), "2018-04-01 00:00:00.000000")) range-filter on: and(ge(scan-collection($$254.getField("o_orderline")).getField("ol_delivery_d"), "2018-01-01 00:00:00.000000"), lt(scan-collection($$254.getField("o_orderline")).getField("ol_delivery_d"), "2018-04-01 00:00:00.000000")) [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|
-                                                                          replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                          -- REPLICATE  |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 [$$268]  |PARTITIONED|
+                            assign [$$268] <- [get-item($$213, 0)] project: [$$268] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- ASSIGN  |UNPARTITIONED|
+                              aggregate [$$213] <- [listify($$278)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- AGGREGATE  |UNPARTITIONED|
+                                aggregate [$$278] <- [agg-global-sql-max($$281)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- AGGREGATE  |UNPARTITIONED|
+                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                    aggregate [$$281] <- [agg-local-sql-max($$210)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- AGGREGATE  |PARTITIONED|
+                                      assign [$$210] <- [{"supplier_no": $$247, "total_rev": $$265}.getField("total_revenue")] project: [$$210] [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]
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          group by ([$$247 := $$283]) decor ([]) {
+                                                    aggregate [$$265] <- [agg-global-sql-sum($$282)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                    -- AGGREGATE  |LOCAL|
+                                                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- SORT_GROUP_BY[$$283]  |PARTITIONED|
+                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                            -- HASH_PARTITION_EXCHANGE [$$283]  |PARTITIONED|
+                                              group by ([$$283 := $$249]) decor ([]) {
+                                                        aggregate [$$282] <- [agg-local-sql-sum($$259)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                        -- AGGREGATE  |LOCAL|
+                                                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- SORT_GROUP_BY[$$249]  |PARTITIONED|
+                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  assign [$$249] <- [numeric-mod(numeric-multiply($$250, $$251), 10000)] project: [$$259, $$249] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    project ([$$250, $$251, $$259]) [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|
+                                                        join (and(eq($$266, $$251), eq($$267, $$250))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                        -- HYBRID_HASH_JOIN [$$251, $$250][$$266, $$267]  |PARTITIONED|
+                                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                            -- REPLICATE  |PARTITIONED|
+                                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                              -- HASH_PARTITION_EXCHANGE [$$251, $$250]  |PARTITIONED|
+                                                                assign [$$251, $$250] <- [$$253.getField("s_i_id"), $$253.getField("s_w_id")] project: [$$250, $$251] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  project ([$$253]) [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 []<-[$$258, $$253] <- test.stock project ({s_w_id:any,s_i_id:any}) [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 [$$266, $$267]  |PARTITIONED|
+                                                            select (and(lt($$256, "2018-04-01 00:00:00.000000"), ge($$256, "2018-01-01 00:00:00.000000"))) project: [$$259, $$266, $$267] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                -- REPLICATE  |PARTITIONED|
+                                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    assign [$$259, $$267, $$266, $$256] <- [$$255.getField("ol_amount"), $$255.getField("ol_supply_w_id"), $$255.getField("ol_i_id"), $$255.getField("ol_delivery_d")] project: [$$259, $$267, $$266, $$256] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      unnest $$255 <- scan-collection($$270) project: [$$255] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                      -- UNNEST  |PARTITIONED|
+                                                                        assign [$$270] <- [$$254.getField("o_orderline")] project: [$$270] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          project ([$$254]) [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|
-                                                                              project ([$$259, $$267, $$266, $$256]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                assign [$$259, $$267, $$266, $$256] <- [$$255.getField("ol_amount"), $$255.getField("ol_supply_w_id"), $$255.getField("ol_i_id"), $$255.getField("ol_delivery_d")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                  project ([$$255]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                    unnest $$255 <- scan-collection($$270) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                    -- UNNEST  |PARTITIONED|
-                                                                                      project ([$$270]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                        assign [$$270] <- [$$254.getField("o_orderline")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                        -- ASSIGN  |PARTITIONED|
-                                                                                          project ([$$254]) [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 []<-[$$257, $$254] <- test.orders project ({o_orderline:[{ol_delivery_d:any,ol_i_id:any,ol_supply_w_id:any,ol_amount:any}]}) filter on: and(ge(scan-collection($$254.getField("o_orderline")).getField("ol_delivery_d"), "2018-01-01 00:00:00.000000"), lt(scan-collection($$254.getField("o_orderline")).getField("ol_delivery_d"), "2018-04-01 00:00:00.000000")) range-filter on: and(ge(scan-collection($$254.getField("o_orderline")).getField("ol_delivery_d"), "2018-01-01 00:00:00.000000"), lt(scan-collection($$254.getField("o_orderline")).getField("ol_delivery_d"), "2018-04-01 00:00:00.000000")) [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 [$$230]  |PARTITIONED|
-                      project ([$$230, $$272, $$273, $$274]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                              data-scan []<-[$$257, $$254] <- test.orders project ({o_orderline:[{ol_delivery_d:any,ol_i_id:any,ol_supply_w_id:any,ol_amount:any}]}) filter on: and(ge(scan-collection($$254.getField("o_orderline")).getField("ol_delivery_d"), "2018-01-01 00:00:00.000000"), lt(scan-collection($$254.getField("o_orderline")).getField("ol_delivery_d"), "2018-04-01 00:00:00.000000")) range-filter on: and(ge(scan-collection($$254.getField("o_orderline")).getField("ol_delivery_d"), "2018-01-01 00:00:00.000000"), lt(scan-collection($$254.getField("o_orderline")).getField("ol_delivery_d"), "2018-04-01 00:00:00.000000")) [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 [$$230]  |PARTITIONED|
+                    assign [$$274, $$273, $$272, $$230] <- [$$su.getField("su_phone"), $$su.getField("su_address"), $$su.getField("su_name"), $$su.getField("su_suppkey")] project: [$$230, $$272, $$273, $$274] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ASSIGN  |PARTITIONED|
+                      project ([$$su]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$274, $$273, $$272, $$230] <- [$$su.getField("su_phone"), $$su.getField("su_address"), $$su.getField("su_name"), $$su.getField("su_suppkey")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ASSIGN  |PARTITIONED|
-                          project ([$$su]) [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 []<-[$$234, $$su] <- test.supplier project ({su_phone:any,su_suppkey:any,su_name:any,su_address:any}) [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|
-                              data-scan []<-[$$234, $$su] <- test.supplier project ({su_phone:any,su_suppkey:any,su_name:any,su_address:any}) [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|
+                              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/replicate/replicate.020.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.020.plan
index 1c34eac..dba3a54 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.020.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.020.plan
@@ -2,69 +2,61 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$54]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$54] <- [{"$1": $$59}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$59] <- [agg-sql-sum($$62)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$62] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                join (eq($$57, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- HYBRID_HASH_JOIN [$$57][$$58]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HASH_PARTITION_EXCHANGE [$$57]  |PARTITIONED|
-                    project ([$$57]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      select (gt($$o1.getField("o_ol_cnt"), 4)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_SELECT  |PARTITIONED|
-                        project ([$$o1, $$57]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$o1, $$57] <- [$$o2, $$58] [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]
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- REPLICATE  |PARTITIONED|
+    assign [$$54] <- [{"$1": $$59}] project: [$$54] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$59] <- [agg-sql-sum($$62)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$62] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              join (eq($$57, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- HYBRID_HASH_JOIN [$$57][$$58]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HASH_PARTITION_EXCHANGE [$$57]  |PARTITIONED|
+                  select (gt($$o1.getField("o_ol_cnt"), 4)) project: [$$57] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_SELECT  |PARTITIONED|
+                    assign [$$o1, $$57] <- [$$o2, $$58] project: [$$o1, $$57] [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]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- REPLICATE  |PARTITIONED|
+                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            assign [$$58] <- [$$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- ASSIGN  |PARTITIONED|
+                              project ([$$o2]) [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|
-                                  assign [$$58] <- [$$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ASSIGN  |PARTITIONED|
-                                    project ([$$o2]) [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 []<-[$$56, $$o2] <- test.orders project ({o_carrier_id:any,o_ol_cnt:any}) filter on: or(gt($$o2.getField("o_ol_cnt"), 4), eq($$o2.getField("o_ol_cnt"), 1)) range-filter on: or(gt($$o2.getField("o_ol_cnt"), 4), eq($$o2.getField("o_ol_cnt"), 1)) [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 [$$58]  |PARTITIONED|
-                    project ([$$58]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      select (eq($$o2.getField("o_ol_cnt"), 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_SELECT  |PARTITIONED|
+                                  data-scan []<-[$$56, $$o2] <- test.orders project ({o_carrier_id:any,o_ol_cnt:any}) filter on: or(gt($$o2.getField("o_ol_cnt"), 4), eq($$o2.getField("o_ol_cnt"), 1)) range-filter on: or(gt($$o2.getField("o_ol_cnt"), 4), eq($$o2.getField("o_ol_cnt"), 1)) [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 [$$58]  |PARTITIONED|
+                  select (eq($$o2.getField("o_ol_cnt"), 1)) project: [$$58] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_SELECT  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- REPLICATE  |PARTITIONED|
                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- REPLICATE  |PARTITIONED|
-                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              assign [$$58] <- [$$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ASSIGN  |PARTITIONED|
-                                project ([$$o2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_PROJECT  |PARTITIONED|
+                          assign [$$58] <- [$$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- ASSIGN  |PARTITIONED|
+                            project ([$$o2]) [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 []<-[$$56, $$o2] <- test.orders project ({o_carrier_id:any,o_ol_cnt:any}) filter on: or(gt($$o2.getField("o_ol_cnt"), 4), eq($$o2.getField("o_ol_cnt"), 1)) range-filter on: or(gt($$o2.getField("o_ol_cnt"), 4), eq($$o2.getField("o_ol_cnt"), 1)) [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|
-                                    data-scan []<-[$$56, $$o2] <- test.orders project ({o_carrier_id:any,o_ol_cnt:any}) filter on: or(gt($$o2.getField("o_ol_cnt"), 4), eq($$o2.getField("o_ol_cnt"), 1)) range-filter on: or(gt($$o2.getField("o_ol_cnt"), 4), eq($$o2.getField("o_ol_cnt"), 1)) [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|
+                                    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/replicate/replicate.021.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.021.plan
index 41e6763..c168866 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.021.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.021.plan
@@ -2,86 +2,76 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$68]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$68] <- [{"$1": $$71}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$71] <- [agg-sql-sum($$78)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$78] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                join (eq($$72, $$73)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- HYBRID_HASH_JOIN [$$72][$$73]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HASH_PARTITION_EXCHANGE [$$72]  |PARTITIONED|
-                    project ([$$72]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      select (eq($$o1.getField("o_orderline").getField("ol_dist_info"), "x")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_SELECT  |PARTITIONED|
-                        assign [$$72] <- [$$o1.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ASSIGN  |PARTITIONED|
-                          project ([$$o1]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            assign [$$o1] <- [$$o2] [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]
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- REPLICATE  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    project ([$$o2]) [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 []<-[$$70, $$o2] <- test.orders project ({o_carrier_id:any,o_orderline:<[{ol_dist_info:any}],{ol_dist_info:any}>}) filter on: or(eq($$o2.getField("o_orderline").getField("ol_dist_info"), "x"), eq(scan-collection($$o2.getField("o_orderline")).getField("ol_dist_info"), "x")) range-filter on: or(eq($$o2.getField("o_orderline").getField("ol_dist_info"), "x"), eq(scan-collection($$o2.getField("o_orderline")).getField("ol_dist_info"), "x")) [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 [$$73]  |PARTITIONED|
-                    project ([$$73]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      select ($$57) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_SELECT  |PARTITIONED|
-                        project ([$$73, $$57]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          subplan {
-                                    aggregate [$$57] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- AGGREGATE  |LOCAL|
-                                      select (eq($$76, "x")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- STREAM_SELECT  |LOCAL|
-                                        assign [$$76] <- [$$ol.getField("ol_dist_info")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- ASSIGN  |LOCAL|
-                                          unnest $$ol <- scan-collection($$75) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- SUBPLAN  |PARTITIONED|
-                            project ([$$73, $$75]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              assign [$$73, $$75] <- [$$o2.getField("o_carrier_id"), $$o2.getField("o_orderline")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ASSIGN  |PARTITIONED|
+    assign [$$68] <- [{"$1": $$71}] project: [$$68] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$71] <- [agg-sql-sum($$78)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$78] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              join (eq($$72, $$73)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- HYBRID_HASH_JOIN [$$72][$$73]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HASH_PARTITION_EXCHANGE [$$72]  |PARTITIONED|
+                  select (eq($$o1.getField("o_orderline").getField("ol_dist_info"), "x")) project: [$$72] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_SELECT  |PARTITIONED|
+                    assign [$$72] <- [$$o1.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ASSIGN  |PARTITIONED|
+                      assign [$$o1] <- [$$o2] project: [$$o1] [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]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- REPLICATE  |PARTITIONED|
+                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              project ([$$o2]) [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|
-                                  replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- REPLICATE  |PARTITIONED|
+                                  data-scan []<-[$$70, $$o2] <- test.orders project ({o_carrier_id:any,o_orderline:<[{ol_dist_info:any}],{ol_dist_info:any}>}) filter on: or(eq($$o2.getField("o_orderline").getField("ol_dist_info"), "x"), eq(scan-collection($$o2.getField("o_orderline")).getField("ol_dist_info"), "x")) range-filter on: or(eq($$o2.getField("o_orderline").getField("ol_dist_info"), "x"), eq(scan-collection($$o2.getField("o_orderline")).getField("ol_dist_info"), "x")) [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|
-                                      project ([$$o2]) [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 []<-[$$70, $$o2] <- test.orders project ({o_carrier_id:any,o_orderline:<[{ol_dist_info:any}],{ol_dist_info:any}>}) filter on: or(eq($$o2.getField("o_orderline").getField("ol_dist_info"), "x"), eq(scan-collection($$o2.getField("o_orderline")).getField("ol_dist_info"), "x")) range-filter on: or(eq($$o2.getField("o_orderline").getField("ol_dist_info"), "x"), eq(scan-collection($$o2.getField("o_orderline")).getField("ol_dist_info"), "x")) [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|
+                                      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 [$$73]  |PARTITIONED|
+                  select ($$57) project: [$$73] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_SELECT  |PARTITIONED|
+                    project ([$$73, $$57]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      subplan {
+                                aggregate [$$57] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- AGGREGATE  |LOCAL|
+                                  select (eq($$76, "x")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- STREAM_SELECT  |LOCAL|
+                                    assign [$$76] <- [$$ol.getField("ol_dist_info")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- ASSIGN  |LOCAL|
+                                      unnest $$ol <- scan-collection($$75) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- SUBPLAN  |PARTITIONED|
+                        assign [$$73, $$75] <- [$$o2.getField("o_carrier_id"), $$o2.getField("o_orderline")] project: [$$73, $$75] [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]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- REPLICATE  |PARTITIONED|
+                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                project ([$$o2]) [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 []<-[$$70, $$o2] <- test.orders project ({o_carrier_id:any,o_orderline:<[{ol_dist_info:any}],{ol_dist_info:any}>}) filter on: or(eq($$o2.getField("o_orderline").getField("ol_dist_info"), "x"), eq(scan-collection($$o2.getField("o_orderline")).getField("ol_dist_info"), "x")) range-filter on: or(eq($$o2.getField("o_orderline").getField("ol_dist_info"), "x"), eq(scan-collection($$o2.getField("o_orderline")).getField("ol_dist_info"), "x")) [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/column/pushdown/replicate/replicate.022.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.022.plan
index d518e0f..11485c0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.022.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.022.plan
@@ -2,65 +2,59 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$52] <- [{"$1": $$57}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$57] <- [agg-sql-sum($$59)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$59] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                join (eq($$55, $$56)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- HYBRID_HASH_JOIN [$$55][$$56]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HASH_PARTITION_EXCHANGE [$$55]  |PARTITIONED|
-                    project ([$$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$o1, $$55] <- [$$o2, $$56] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
+    assign [$$52] <- [{"$1": $$57}] project: [$$52] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$57] <- [agg-sql-sum($$59)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$59] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              join (eq($$55, $$56)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- HYBRID_HASH_JOIN [$$55][$$56]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HASH_PARTITION_EXCHANGE [$$55]  |PARTITIONED|
+                  assign [$$o1, $$55] <- [$$o2, $$56] project: [$$55] [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]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- REPLICATE  |PARTITIONED|
                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- REPLICATE  |PARTITIONED|
-                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              assign [$$56] <- [$$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ASSIGN  |PARTITIONED|
-                                project ([$$o2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_PROJECT  |PARTITIONED|
+                          assign [$$56] <- [$$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- ASSIGN  |PARTITIONED|
+                            project ([$$o2]) [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 []<-[$$54, $$o2] <- test.orders project ({o_carrier_id:any,o_ol_cnt:any}) [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|
-                                    data-scan []<-[$$54, $$o2] <- test.orders project ({o_carrier_id:any,o_ol_cnt:any}) [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 [$$56]  |PARTITIONED|
-                    project ([$$56]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      select (eq($$o2.getField("o_ol_cnt"), 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_SELECT  |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 [$$56]  |PARTITIONED|
+                  select (eq($$o2.getField("o_ol_cnt"), 1)) project: [$$56] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_SELECT  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- REPLICATE  |PARTITIONED|
                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- REPLICATE  |PARTITIONED|
-                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              assign [$$56] <- [$$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ASSIGN  |PARTITIONED|
-                                project ([$$o2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_PROJECT  |PARTITIONED|
+                          assign [$$56] <- [$$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- ASSIGN  |PARTITIONED|
+                            project ([$$o2]) [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 []<-[$$54, $$o2] <- test.orders project ({o_carrier_id:any,o_ol_cnt:any}) [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|
-                                    data-scan []<-[$$54, $$o2] <- test.orders project ({o_carrier_id:any,o_ol_cnt:any}) [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|
+                                    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/replicate/replicate.023.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.023.plan
index 49a699f..28afa1c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.023.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.023.plan
@@ -2,57 +2,29 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$54]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$54] <- [{"$1": $$59}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$59] <- [agg-sql-sum($$62)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$62] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                join (eq($$57, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- HYBRID_HASH_JOIN [$$57][$$58]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$57]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$57] <- [$$58] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
+    assign [$$54] <- [{"$1": $$59}] project: [$$54] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$59] <- [agg-sql-sum($$62)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$62] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              join (eq($$57, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- HYBRID_HASH_JOIN [$$57][$$58]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  assign [$$57] <- [$$58] project: [$$57] [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]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- REPLICATE  |PARTITIONED|
                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- REPLICATE  |PARTITIONED|
-                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-                              project ([$$58]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                select (eq($$o2.getField("o_ol_cnt"), 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  assign [$$58] <- [$$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ASSIGN  |PARTITIONED|
-                                    project ([$$o2]) [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 []<-[$$56, $$o2] <- test.orders project ({o_carrier_id:any,o_ol_cnt:any}) filter on: eq($$o2.getField("o_ol_cnt"), 1) range-filter on: eq($$o2.getField("o_ol_cnt"), 1) [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  |PARTITIONED|
-                    replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- REPLICATE  |PARTITIONED|
-                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-                        project ([$$58]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          select (eq($$o2.getField("o_ol_cnt"), 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
+                          select (eq($$o2.getField("o_ol_cnt"), 1)) project: [$$58] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                           -- STREAM_SELECT  |PARTITIONED|
                             assign [$$58] <- [$$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- ASSIGN  |PARTITIONED|
@@ -66,3 +38,23 @@
                                     -- 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  |PARTITIONED|
+                  replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- REPLICATE  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
+                      select (eq($$o2.getField("o_ol_cnt"), 1)) project: [$$58] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- STREAM_SELECT  |PARTITIONED|
+                        assign [$$58] <- [$$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- ASSIGN  |PARTITIONED|
+                          project ([$$o2]) [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 []<-[$$56, $$o2] <- test.orders project ({o_carrier_id:any,o_ol_cnt:any}) filter on: eq($$o2.getField("o_ol_cnt"), 1) range-filter on: eq($$o2.getField("o_ol_cnt"), 1) [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/column/pushdown/replicate/replicate.030.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.030.plan
index c978be9..5c6765e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.030.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.030.plan
@@ -2,63 +2,57 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$34]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$34] <- [{"o1": $$o1}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$o1]) [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|
-            join (eq($$37, $$38)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HYBRID_HASH_JOIN [$$37][$$38]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
-                select (gt($$o1.getField("o_ol_cnt"), 4)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  project ([$$o1, $$37]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$o1, $$37] <- [$$o2, $$38] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
+    assign [$$34] <- [{"o1": $$o1}] project: [$$34] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$o1]) [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|
+          join (eq($$37, $$38)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HYBRID_HASH_JOIN [$$37][$$38]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+              select (gt($$o1.getField("o_ol_cnt"), 4)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                assign [$$o1, $$37] <- [$$o2, $$38] project: [$$o1, $$37] [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]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- REPLICATE  |PARTITIONED|
                       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- REPLICATE  |PARTITIONED|
-                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            assign [$$38] <- [$$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- ASSIGN  |PARTITIONED|
-                              project ([$$o2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
+                        assign [$$38] <- [$$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- ASSIGN  |PARTITIONED|
+                          project ([$$o2]) [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 []<-[$$36, $$o2] <- test.orders filter on: or(gt($$o2.getField("o_ol_cnt"), 4), eq($$o2.getField("o_ol_cnt"), 1)) range-filter on: or(gt($$o2.getField("o_ol_cnt"), 4), eq($$o2.getField("o_ol_cnt"), 1)) [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|
-                                  data-scan []<-[$$36, $$o2] <- test.orders filter on: or(gt($$o2.getField("o_ol_cnt"), 4), eq($$o2.getField("o_ol_cnt"), 1)) range-filter on: or(gt($$o2.getField("o_ol_cnt"), 4), eq($$o2.getField("o_ol_cnt"), 1)) [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 [$$38]  |PARTITIONED|
-                project ([$$38]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  select (eq($$o2.getField("o_ol_cnt"), 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_SELECT  |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 [$$38]  |PARTITIONED|
+              select (eq($$o2.getField("o_ol_cnt"), 1)) project: [$$38] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- REPLICATE  |PARTITIONED|
                     exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- REPLICATE  |PARTITIONED|
-                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          assign [$$38] <- [$$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- ASSIGN  |PARTITIONED|
-                            project ([$$o2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
+                      assign [$$38] <- [$$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- ASSIGN  |PARTITIONED|
+                        project ([$$o2]) [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 []<-[$$36, $$o2] <- test.orders filter on: or(gt($$o2.getField("o_ol_cnt"), 4), eq($$o2.getField("o_ol_cnt"), 1)) range-filter on: or(gt($$o2.getField("o_ol_cnt"), 4), eq($$o2.getField("o_ol_cnt"), 1)) [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|
-                                data-scan []<-[$$36, $$o2] <- test.orders filter on: or(gt($$o2.getField("o_ol_cnt"), 4), eq($$o2.getField("o_ol_cnt"), 1)) range-filter on: or(gt($$o2.getField("o_ol_cnt"), 4), eq($$o2.getField("o_ol_cnt"), 1)) [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|
+                                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/replicate/replicate.031.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.031.plan
index 90d57ef..9c1ea1e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.031.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.031.plan
@@ -2,65 +2,57 @@
 -- 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]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$36] <- [{"o1_id": $$43, "o2_id": $$44}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$43, $$44]) [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|
-            join (eq($$39, $$40)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HYBRID_HASH_JOIN [$$39][$$40]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
-                project ([$$43, $$39]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  select (gt($$o1.getField("o_ol_cnt"), 4)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_SELECT  |PARTITIONED|
-                    project ([$$o1, $$43, $$39]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$o1, $$43, $$39] <- [$$o2, $$44, $$40] [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]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- REPLICATE  |PARTITIONED|
+    assign [$$36] <- [{"o1_id": $$43, "o2_id": $$44}] project: [$$36] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$43, $$44]) [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|
+          join (eq($$39, $$40)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HYBRID_HASH_JOIN [$$39][$$40]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
+              select (gt($$o1.getField("o_ol_cnt"), 4)) project: [$$43, $$39] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                assign [$$o1, $$43, $$39] <- [$$o2, $$44, $$40] project: [$$o1, $$43, $$39] [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]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- REPLICATE  |PARTITIONED|
+                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        assign [$$44, $$40] <- [$$o2.getField("o_id"), $$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- ASSIGN  |PARTITIONED|
+                          project ([$$o2]) [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|
-                              assign [$$44, $$40] <- [$$o2.getField("o_id"), $$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ASSIGN  |PARTITIONED|
-                                project ([$$o2]) [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 []<-[$$38, $$o2] <- test.orders project ({o_carrier_id:any,o_id:any,o_ol_cnt:any}) filter on: or(gt($$o2.getField("o_ol_cnt"), 4), eq($$o2.getField("o_ol_cnt"), 1)) range-filter on: or(gt($$o2.getField("o_ol_cnt"), 4), eq($$o2.getField("o_ol_cnt"), 1)) [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 [$$40]  |PARTITIONED|
-                project ([$$44, $$40]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  select (eq($$o2.getField("o_ol_cnt"), 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_SELECT  |PARTITIONED|
+                              data-scan []<-[$$38, $$o2] <- test.orders project ({o_carrier_id:any,o_id:any,o_ol_cnt:any}) filter on: or(gt($$o2.getField("o_ol_cnt"), 4), eq($$o2.getField("o_ol_cnt"), 1)) range-filter on: or(gt($$o2.getField("o_ol_cnt"), 4), eq($$o2.getField("o_ol_cnt"), 1)) [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 [$$40]  |PARTITIONED|
+              select (eq($$o2.getField("o_ol_cnt"), 1)) project: [$$44, $$40] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- REPLICATE  |PARTITIONED|
                     exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- REPLICATE  |PARTITIONED|
-                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          assign [$$44, $$40] <- [$$o2.getField("o_id"), $$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- ASSIGN  |PARTITIONED|
-                            project ([$$o2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
+                      assign [$$44, $$40] <- [$$o2.getField("o_id"), $$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- ASSIGN  |PARTITIONED|
+                        project ([$$o2]) [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 []<-[$$38, $$o2] <- test.orders project ({o_carrier_id:any,o_id:any,o_ol_cnt:any}) filter on: or(gt($$o2.getField("o_ol_cnt"), 4), eq($$o2.getField("o_ol_cnt"), 1)) range-filter on: or(gt($$o2.getField("o_ol_cnt"), 4), eq($$o2.getField("o_ol_cnt"), 1)) [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|
-                                data-scan []<-[$$38, $$o2] <- test.orders project ({o_carrier_id:any,o_id:any,o_ol_cnt:any}) filter on: or(gt($$o2.getField("o_ol_cnt"), 4), eq($$o2.getField("o_ol_cnt"), 1)) range-filter on: or(gt($$o2.getField("o_ol_cnt"), 4), eq($$o2.getField("o_ol_cnt"), 1)) [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|
+                                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/replicate/replicate.032.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.032.plan
index 1513679..e38a990 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.032.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.032.plan
@@ -2,65 +2,57 @@
 -- 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]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$36] <- [{"o1_id": $$43, "o2_id": $$44}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$43, $$44]) [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|
-            join (eq($$39, $$40)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HYBRID_HASH_JOIN [$$39][$$40]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
-                project ([$$43, $$39]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  select (gt($$o1.getField("o_ol_cnt_1"), 4)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_SELECT  |PARTITIONED|
-                    project ([$$o1, $$43, $$39]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$o1, $$43, $$39] <- [$$o2, $$44, $$40] [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]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- REPLICATE  |PARTITIONED|
+    assign [$$36] <- [{"o1_id": $$43, "o2_id": $$44}] project: [$$36] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$43, $$44]) [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|
+          join (eq($$39, $$40)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HYBRID_HASH_JOIN [$$39][$$40]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
+              select (gt($$o1.getField("o_ol_cnt_1"), 4)) project: [$$43, $$39] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                assign [$$o1, $$43, $$39] <- [$$o2, $$44, $$40] project: [$$o1, $$43, $$39] [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]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- REPLICATE  |PARTITIONED|
+                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        assign [$$44, $$40] <- [$$o2.getField("o_id"), $$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- ASSIGN  |PARTITIONED|
+                          project ([$$o2]) [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|
-                              assign [$$44, $$40] <- [$$o2.getField("o_id"), $$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ASSIGN  |PARTITIONED|
-                                project ([$$o2]) [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 []<-[$$38, $$o2] <- test.orders project ({o_carrier_id:any,o_id:any,o_ol_cnt_2:any,o_ol_cnt_1:any}) filter on: or(gt($$o2.getField("o_ol_cnt_1"), 4), eq($$o2.getField("o_ol_cnt_2"), 1)) range-filter on: or(gt($$o2.getField("o_ol_cnt_1"), 4), eq($$o2.getField("o_ol_cnt_2"), 1)) [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 [$$40]  |PARTITIONED|
-                project ([$$44, $$40]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  select (eq($$o2.getField("o_ol_cnt_2"), 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_SELECT  |PARTITIONED|
+                              data-scan []<-[$$38, $$o2] <- test.orders project ({o_carrier_id:any,o_id:any,o_ol_cnt_2:any,o_ol_cnt_1:any}) filter on: or(gt($$o2.getField("o_ol_cnt_1"), 4), eq($$o2.getField("o_ol_cnt_2"), 1)) range-filter on: or(gt($$o2.getField("o_ol_cnt_1"), 4), eq($$o2.getField("o_ol_cnt_2"), 1)) [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 [$$40]  |PARTITIONED|
+              select (eq($$o2.getField("o_ol_cnt_2"), 1)) project: [$$44, $$40] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- REPLICATE  |PARTITIONED|
                     exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- REPLICATE  |PARTITIONED|
-                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          assign [$$44, $$40] <- [$$o2.getField("o_id"), $$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- ASSIGN  |PARTITIONED|
-                            project ([$$o2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
+                      assign [$$44, $$40] <- [$$o2.getField("o_id"), $$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- ASSIGN  |PARTITIONED|
+                        project ([$$o2]) [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 []<-[$$38, $$o2] <- test.orders project ({o_carrier_id:any,o_id:any,o_ol_cnt_2:any,o_ol_cnt_1:any}) filter on: or(gt($$o2.getField("o_ol_cnt_1"), 4), eq($$o2.getField("o_ol_cnt_2"), 1)) range-filter on: or(gt($$o2.getField("o_ol_cnt_1"), 4), eq($$o2.getField("o_ol_cnt_2"), 1)) [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|
-                                data-scan []<-[$$38, $$o2] <- test.orders project ({o_carrier_id:any,o_id:any,o_ol_cnt_2:any,o_ol_cnt_1:any}) filter on: or(gt($$o2.getField("o_ol_cnt_1"), 4), eq($$o2.getField("o_ol_cnt_2"), 1)) range-filter on: or(gt($$o2.getField("o_ol_cnt_1"), 4), eq($$o2.getField("o_ol_cnt_2"), 1)) [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|
+                                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/replicate/replicate.033.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.033.plan
index 1d45d6e..871b3af 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.033.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.033.plan
@@ -2,46 +2,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$34]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$34] <- [{"o1": $$o1, "o2": $$o2}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$o1, $$o2]) [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|
-            join (eq($$37, $$38)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HYBRID_HASH_JOIN [$$37][$$38]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
-                select (gt($$o1.getField("o_ol_cnt_1"), 4)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  project ([$$o1, $$37]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$o1, $$37] <- [$$o2, $$38] [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]
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- REPLICATE  |PARTITIONED|
-                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            assign [$$38] <- [$$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- ASSIGN  |PARTITIONED|
-                              project ([$$o2]) [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 []<-[$$36, $$o2] <- test.orders filter on: or(gt($$o2.getField("o_ol_cnt_1"), 4), eq($$o2.getField("o_ol_cnt_2"), 1)) range-filter on: or(gt($$o2.getField("o_ol_cnt_1"), 4), eq($$o2.getField("o_ol_cnt_2"), 1)) [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 [$$38]  |PARTITIONED|
-                select (eq($$o2.getField("o_ol_cnt_2"), 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
+    assign [$$34] <- [{"o1": $$o1, "o2": $$o2}] project: [$$34] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$o1, $$o2]) [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|
+          join (eq($$37, $$38)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HYBRID_HASH_JOIN [$$37][$$38]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+              select (gt($$o1.getField("o_ol_cnt_1"), 4)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                assign [$$o1, $$37] <- [$$o2, $$38] project: [$$o1, $$37] [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]
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
@@ -60,3 +34,25 @@
                                 -- 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 [$$38]  |PARTITIONED|
+              select (eq($$o2.getField("o_ol_cnt_2"), 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- REPLICATE  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      assign [$$38] <- [$$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- ASSIGN  |PARTITIONED|
+                        project ([$$o2]) [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 []<-[$$36, $$o2] <- test.orders filter on: or(gt($$o2.getField("o_ol_cnt_1"), 4), eq($$o2.getField("o_ol_cnt_2"), 1)) range-filter on: or(gt($$o2.getField("o_ol_cnt_1"), 4), eq($$o2.getField("o_ol_cnt_2"), 1)) [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/column/pushdown/replicate/replicate.034.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.034.plan
index 8c67a3f..41aefee 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.034.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/replicate/replicate.034.plan
@@ -2,69 +2,61 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$39]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$39] <- [{"o1_dist_info": $$46, "o2_dist_info": $$47}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$46, $$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|
-            join (eq($$42, $$43)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HYBRID_HASH_JOIN [$$42][$$43]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
-                project ([$$46, $$42]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$46] <- [$$o1.getField("o_orderline").getField("ol_dist_info")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$39] <- [{"o1_dist_info": $$46, "o2_dist_info": $$47}] project: [$$39] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$46, $$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|
+          join (eq($$42, $$43)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HYBRID_HASH_JOIN [$$42][$$43]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
+              assign [$$46] <- [$$o1.getField("o_orderline").getField("ol_dist_info")] project: [$$46, $$42] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ASSIGN  |PARTITIONED|
+                select (gt($$o1.getField("o_ol_cnt_1"), 4)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STREAM_SELECT  |PARTITIONED|
+                  assign [$$o1, $$42] <- [$$o2, $$43] project: [$$o1, $$42] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- ASSIGN  |PARTITIONED|
-                    select (gt($$o1.getField("o_ol_cnt_1"), 4)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_SELECT  |PARTITIONED|
-                      project ([$$o1, $$42]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$o1, $$42] <- [$$o2, $$43] [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]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- REPLICATE  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- REPLICATE  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          assign [$$43] <- [$$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- ASSIGN  |PARTITIONED|
+                            project ([$$o2]) [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|
-                                assign [$$43] <- [$$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- ASSIGN  |PARTITIONED|
-                                  project ([$$o2]) [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, $$o2] <- test.orders project ({o_carrier_id:any,o_ol_cnt_2:any,o_ol_cnt_1:any,o_orderline:<[{ol_dist_info:any}],{ol_dist_info:any}>}) filter on: or(gt($$o2.getField("o_ol_cnt_1"), 4), eq($$o2.getField("o_ol_cnt_2"), 1)) range-filter on: or(gt($$o2.getField("o_ol_cnt_1"), 4), eq($$o2.getField("o_ol_cnt_2"), 1)) [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 [$$43]  |PARTITIONED|
-                project ([$$47, $$43]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$47] <- [get-item($$o2.getField("o_orderline"), 0).getField("ol_dist_info")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ASSIGN  |PARTITIONED|
-                    select (eq($$o2.getField("o_ol_cnt_2"), 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_SELECT  |PARTITIONED|
+                                data-scan []<-[$$41, $$o2] <- test.orders project ({o_carrier_id:any,o_ol_cnt_2:any,o_ol_cnt_1:any,o_orderline:<[{ol_dist_info:any}],{ol_dist_info:any}>}) filter on: or(gt($$o2.getField("o_ol_cnt_1"), 4), eq($$o2.getField("o_ol_cnt_2"), 1)) range-filter on: or(gt($$o2.getField("o_ol_cnt_1"), 4), eq($$o2.getField("o_ol_cnt_2"), 1)) [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 [$$43]  |PARTITIONED|
+              assign [$$47] <- [get-item($$o2.getField("o_orderline"), 0).getField("ol_dist_info")] project: [$$47, $$43] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ASSIGN  |PARTITIONED|
+                select (eq($$o2.getField("o_ol_cnt_2"), 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STREAM_SELECT  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- REPLICATE  |PARTITIONED|
                       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- REPLICATE  |PARTITIONED|
-                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            assign [$$43] <- [$$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- ASSIGN  |PARTITIONED|
-                              project ([$$o2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
+                        assign [$$43] <- [$$o2.getField("o_carrier_id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- ASSIGN  |PARTITIONED|
+                          project ([$$o2]) [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, $$o2] <- test.orders project ({o_carrier_id:any,o_ol_cnt_2:any,o_ol_cnt_1:any,o_orderline:<[{ol_dist_info:any}],{ol_dist_info:any}>}) filter on: or(gt($$o2.getField("o_ol_cnt_1"), 4), eq($$o2.getField("o_ol_cnt_2"), 1)) range-filter on: or(gt($$o2.getField("o_ol_cnt_1"), 4), eq($$o2.getField("o_ol_cnt_2"), 1)) [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|
-                                  data-scan []<-[$$41, $$o2] <- test.orders project ({o_carrier_id:any,o_ol_cnt_2:any,o_ol_cnt_1:any,o_orderline:<[{ol_dist_info:any}],{ol_dist_info:any}>}) filter on: or(gt($$o2.getField("o_ol_cnt_1"), 4), eq($$o2.getField("o_ol_cnt_2"), 1)) range-filter on: or(gt($$o2.getField("o_ol_cnt_1"), 4), eq($$o2.getField("o_ol_cnt_2"), 1)) [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|
+                                  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/copy-to-hdfs/parquet-cover-data-types/parquet-cover-data-types.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to-hdfs/parquet-cover-data-types/parquet-cover-data-types.05.adm
new file mode 100644
index 0000000..8fc863e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to-hdfs/parquet-cover-data-types/parquet-cover-data-types.05.adm
@@ -0,0 +1 @@
+{ "name": "Virat", "id": 18, "dateType": date("1988-11-05"), "timeType": time("03:10:00.493"), "boolType": false, "doubleType": 0.75, "datetimeType": datetime("1900-02-01T00:00:00.000") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to-hdfs/parquet-empty-array/parquet-empty-array.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to-hdfs/parquet-empty-array/parquet-empty-array.05.adm
new file mode 100644
index 0000000..97de7e9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to-hdfs/parquet-empty-array/parquet-empty-array.05.adm
@@ -0,0 +1,2 @@
+{ "id": 2, "name": { "first": [ "power", "star" ] } }
+{ "id": 5, "name": { "first": [  ] } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to-hdfs/parquet-simple/parquet-simple.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to-hdfs/parquet-simple/parquet-simple.04.adm
new file mode 100644
index 0000000..bf567b2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to-hdfs/parquet-simple/parquet-simple.04.adm
@@ -0,0 +1 @@
+{ "id": "123" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to-hdfs/parquet-tweet/parquet-tweet.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to-hdfs/parquet-tweet/parquet-tweet.05.adm
new file mode 100644
index 0000000..5e0df96
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to-hdfs/parquet-tweet/parquet-tweet.05.adm
@@ -0,0 +1,2 @@
+{ "coordinates": { "coordinates": [ 1.1 ], "type": "string" }, "created_at": "string", "entities": { "urls": [ { "display_url": "string", "expanded_url": "string", "indices": [ 1 ], "url": "string" } ], "user_mentions": [ { "id": 1, "id_str": "string", "indices": [ 1 ], "name": "string", "screen_name": "string" } ] }, "favorite_count": 1, "favorited": true, "filter_level": "string", "geo": { "coordinates": [ 1.1 ], "type": "string" }, "id": "0000000", "id_str": "string", "in_reply_to_screen_name": "string", "in_reply_to_status_id": 1, "in_reply_to_status_id_str": "string", "in_reply_to_user_id": 1, "in_reply_to_user_id_str": "string", "is_quote_status": true, "lang": "string", "place": { "bounding_box": { "coordinates": [ [ [ 1.1 ] ] ], "type": "string" }, "country": "string", "country_code": "string", "full_name": "string", "id": "string", "name": "string", "place_type": "string", "url": "string" }, "possibly_sensitive": true, "quoted_status": { "created_at": "string", "entities": { "user_mentions": [ { "id": 1, "id_str": "string", "indices": [ 1 ], "name": "string", "screen_name": "string" } ] }, "favorite_count": 1, "favorited": true, "filter_level": "string", "id": 1, "id_str": "string", "in_reply_to_screen_name": "string", "in_reply_to_status_id": 1, "in_reply_to_status_id_str": "string", "in_reply_to_user_id": 1, "in_reply_to_user_id_str": "string", "is_quote_status": true, "lang": "string", "retweet_count": 1, "retweeted": true, "source": "string", "text": "string", "truncated": true, "user": { "contributors_enabled": true, "created_at": "string", "default_profile": true, "default_profile_image": true, "description": "string", "favourites_count": 1, "followers_count": 1, "friends_count": 1, "geo_enabled": true, "id": 1, "id_str": "string", "is_translator": true, "lang": "string", "listed_count": 1, "name": "string", "profile_background_color": "string", "profile_background_image_url": "string", "profile_background_image_url_https": "string", "profile_background_tile": true, "profile_banner_url": "string", "profile_image_url": "string", "profile_image_url_https": "string", "profile_link_color": "string", "profile_sidebar_border_color": "string", "profile_sidebar_fill_color": "string", "profile_text_color": "string", "profile_use_background_image": true, "protected": true, "screen_name": "string", "statuses_count": 1, "verified": true } }, "quoted_status_id": 1, "quoted_status_id_str": "string", "retweet_count": 1, "retweeted": true, "source": "string", "text": "string", "timestamp_ms": "string", "truncated": true, "user": { "contributors_enabled": true, "created_at": "string", "default_profile": true, "default_profile_image": true, "description": "string", "favourites_count": 1, "followers_count": 1, "friends_count": 1, "geo_enabled": true, "id": 1, "id_str": "string", "is_translator": true, "lang": "string", "listed_count": 1, "location": "string", "name": "string", "profile_background_color": "string", "profile_background_image_url": "string", "profile_background_image_url_https": "string", "profile_background_tile": true, "profile_banner_url": "string", "profile_image_url": "string", "profile_image_url_https": "string", "profile_link_color": "string", "profile_sidebar_border_color": "string", "profile_sidebar_fill_color": "string", "profile_text_color": "string", "profile_use_background_image": true, "protected": true, "screen_name": "string", "statuses_count": 1, "time_zone": "string", "url": "string", "utc_offset": 1, "verified": true } }
+{ "coordinates": { "coordinates": [ 1.1 ], "type": "string" }, "created_at": "string", "favorite_count": 1, "favorited": true, "filter_level": "string", "geo": { "coordinates": [ 1.1 ], "type": "string" }, "id": "11111111111111111111", "id_str": "string", "in_reply_to_screen_name": "string", "in_reply_to_status_id": 1, "in_reply_to_status_id_str": "string", "in_reply_to_user_id": 1, "in_reply_to_user_id_str": "string", "is_quote_status": true, "lang": "string", "place": { "bounding_box": { "coordinates": [ [ [ 1.1 ] ] ], "type": "string" }, "country": "string", "country_code": "string", "full_name": "string", "id": "string", "name": "string", "place_type": "string", "url": "string" }, "possibly_sensitive": true, "quoted_status": { "created_at": "string", "entities": { "user_mentions": [ { "id": 1, "id_str": "string", "indices": [ 1 ], "name": "string", "screen_name": "string" } ] }, "favorite_count": 1, "favorited": true, "filter_level": "string", "id": 1, "id_str": "string", "in_reply_to_screen_name": "string", "in_reply_to_status_id": 1, "in_reply_to_status_id_str": "string", "in_reply_to_user_id": 1, "in_reply_to_user_id_str": "string", "is_quote_status": true, "lang": "string", "retweet_count": 1, "retweeted": true, "source": "string", "text": "string", "truncated": true, "user": { "contributors_enabled": true, "created_at": "string", "default_profile": true, "default_profile_image": true, "description": "string", "favourites_count": 1, "followers_count": 1, "friends_count": 1, "geo_enabled": true, "id": 1, "id_str": "string", "is_translator": true, "lang": "string", "listed_count": 1, "name": "string", "profile_background_color": "string", "profile_background_image_url": "string", "profile_background_image_url_https": "string", "profile_background_tile": true, "profile_banner_url": "string", "profile_image_url": "string", "profile_image_url_https": "string", "profile_link_color": "string", "profile_sidebar_border_color": "string", "profile_sidebar_fill_color": "string", "profile_text_color": "string", "profile_use_background_image": true, "protected": true, "screen_name": "string", "statuses_count": 1, "verified": true } }, "quoted_status_id": 1, "quoted_status_id_str": "string", "retweet_count": 1, "retweeted": true, "source": "string", "text": "string", "timestamp_ms": "string", "truncated": true, "user": { "contributors_enabled": true, "created_at": "string", "default_profile": true, "default_profile_image": true, "description": "string", "favourites_count": 1, "followers_count": 1, "friends_count": 1, "geo_enabled": true, "id": 1, "id_str": "string", "is_translator": true, "lang": "string", "listed_count": 1, "location": "string", "name": "string", "profile_background_color": "string", "profile_background_image_url": "string", "profile_background_image_url_https": "string", "profile_background_tile": true, "profile_banner_url": "string", "profile_image_url": "string", "profile_image_url_https": "string", "profile_link_color": "string", "profile_sidebar_border_color": "string", "profile_sidebar_fill_color": "string", "profile_text_color": "string", "profile_use_background_image": true, "protected": true, "screen_name": "string", "statuses_count": 1, "time_zone": "string", "url": "string", "utc_offset": 1, "verified": true } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to-hdfs/parquet-utf8/parquet-utf8.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to-hdfs/parquet-utf8/parquet-utf8.05.adm
new file mode 100644
index 0000000..c60145d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to-hdfs/parquet-utf8/parquet-utf8.05.adm
@@ -0,0 +1,8 @@
+{ "id": 1, "name": "John" }
+{ "id": 2, "name": "Abel" }
+{ "id": 3, "name": "Sandy" }
+{ "id": 4, "name": "Alex" }
+{ "id": 5, "name": "Mike" }
+{ "id": 6, "name": "Tom" }
+{ "comment": "😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا", "id": 7, "name": "Jerry" }
+{ "comment": "😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا", "id": 8, "name": "William" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/delimiter/delimiter.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/delimiter/delimiter.05.adm
new file mode 100644
index 0000000..8722e43
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/delimiter/delimiter.05.adm
@@ -0,0 +1,11 @@
+{ "id": 1, "name": "Macbook1", "amount": 123.2, "accountNumber": 345.34 }
+{ "id": 2, "name": "Macbook2", "amount": 456.7, "accountNumber": 123.45 }
+{ "id": 3, "name": "Macbook3", "amount": 789.1, "accountNumber": 678.90 }
+{ "id": 4, "name": "Macbook4", "amount": 234.5, "accountNumber": 567.89 }
+{ "id": 5, "name": "Macbook5", "amount": 876.5, "accountNumber": 345.67 }
+{ "id": 6, "name": "Macbook6", "amount": 345.6, "accountNumber": 987.65 }
+{ "id": 7, "name": "Macbook7", "amount": 678.9, "accountNumber": 234.56 }
+{ "id": 8, "name": "Macbook8", "amount": 987.2, "accountNumber": 789.12 }
+{ "id": 9, "name": "Macbook9", "amount": 543.2, "accountNumber": 321.45 }
+{ "id": 10, "name": "Macbook10", "amount": 123.9, "accountNumber": 654.32 }
+{ "id": 11, "name": "Macbook11", "amount": 567.8, "accountNumber": 456.78 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/delimiter/delimiter.07.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/delimiter/delimiter.07.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/delimiter/delimiter.07.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/header/header.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/header/header.05.adm
new file mode 100644
index 0000000..8722e43
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/header/header.05.adm
@@ -0,0 +1,11 @@
+{ "id": 1, "name": "Macbook1", "amount": 123.2, "accountNumber": 345.34 }
+{ "id": 2, "name": "Macbook2", "amount": 456.7, "accountNumber": 123.45 }
+{ "id": 3, "name": "Macbook3", "amount": 789.1, "accountNumber": 678.90 }
+{ "id": 4, "name": "Macbook4", "amount": 234.5, "accountNumber": 567.89 }
+{ "id": 5, "name": "Macbook5", "amount": 876.5, "accountNumber": 345.67 }
+{ "id": 6, "name": "Macbook6", "amount": 345.6, "accountNumber": 987.65 }
+{ "id": 7, "name": "Macbook7", "amount": 678.9, "accountNumber": 234.56 }
+{ "id": 8, "name": "Macbook8", "amount": 987.2, "accountNumber": 789.12 }
+{ "id": 9, "name": "Macbook9", "amount": 543.2, "accountNumber": 321.45 }
+{ "id": 10, "name": "Macbook10", "amount": 123.9, "accountNumber": 654.32 }
+{ "id": 11, "name": "Macbook11", "amount": 567.8, "accountNumber": 456.78 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/header/header.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/header/header.12.adm
new file mode 100644
index 0000000..5055d28
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/header/header.12.adm
@@ -0,0 +1,11 @@
+{ "id": 1, "name": "", "amount": 123.2, "accountNumber": 345.34 }
+{ "id": 2, "name": "", "amount": 456.7, "accountNumber": 123.45 }
+{ "id": 3, "name": "", "amount": 789.1, "accountNumber": 678.90 }
+{ "id": 4, "name": "", "amount": 234.5, "accountNumber": 567.89 }
+{ "id": 5, "name": "", "amount": 876.5, "accountNumber": 345.67 }
+{ "id": 6, "name": "", "amount": 345.6, "accountNumber": 987.65 }
+{ "id": 7, "name": "", "amount": 678.9, "accountNumber": 234.56 }
+{ "id": 8, "name": "", "amount": 987.2, "accountNumber": 789.12 }
+{ "id": 9, "name": "", "amount": 543.2, "accountNumber": 321.45 }
+{ "id": 10, "name": "", "amount": 123.9, "accountNumber": 654.32 }
+{ "id": 11, "name": "", "amount": 567.8, "accountNumber": 456.78 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/header/header.22.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/header/header.22.adm
new file mode 100644
index 0000000..90e44c4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/header/header.22.adm
@@ -0,0 +1,11 @@
+{ "id": 1, "name": "IamNull", "amount": 123.2, "accountNumber": 345.34 }
+{ "id": 2, "name": "IamNull", "amount": 456.7, "accountNumber": 123.45 }
+{ "id": 3, "name": "IamNull", "amount": 789.1, "accountNumber": 678.90 }
+{ "id": 4, "name": "IamNull", "amount": 234.5, "accountNumber": 567.89 }
+{ "id": 5, "name": "IamNull", "amount": 876.5, "accountNumber": 345.67 }
+{ "id": 6, "name": "IamNull", "amount": 345.6, "accountNumber": 987.65 }
+{ "id": 7, "name": "IamNull", "amount": 678.9, "accountNumber": 234.56 }
+{ "id": 8, "name": "IamNull", "amount": 987.2, "accountNumber": 789.12 }
+{ "id": 9, "name": "IamNull", "amount": 543.2, "accountNumber": 321.45 }
+{ "id": 10, "name": "IamNull", "amount": 123.9, "accountNumber": 654.32 }
+{ "id": 11, "name": "IamNull", "amount": 567.8, "accountNumber": 456.78 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/header/header.32.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/header/header.32.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/header/header.32.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/null/null.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/null/null.05.adm
new file mode 100644
index 0000000..90e44c4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/null/null.05.adm
@@ -0,0 +1,11 @@
+{ "id": 1, "name": "IamNull", "amount": 123.2, "accountNumber": 345.34 }
+{ "id": 2, "name": "IamNull", "amount": 456.7, "accountNumber": 123.45 }
+{ "id": 3, "name": "IamNull", "amount": 789.1, "accountNumber": 678.90 }
+{ "id": 4, "name": "IamNull", "amount": 234.5, "accountNumber": 567.89 }
+{ "id": 5, "name": "IamNull", "amount": 876.5, "accountNumber": 345.67 }
+{ "id": 6, "name": "IamNull", "amount": 345.6, "accountNumber": 987.65 }
+{ "id": 7, "name": "IamNull", "amount": 678.9, "accountNumber": 234.56 }
+{ "id": 8, "name": "IamNull", "amount": 987.2, "accountNumber": 789.12 }
+{ "id": 9, "name": "IamNull", "amount": 543.2, "accountNumber": 321.45 }
+{ "id": 10, "name": "IamNull", "amount": 123.9, "accountNumber": 654.32 }
+{ "id": 11, "name": "IamNull", "amount": 567.8, "accountNumber": 456.78 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/quote-escape/quote-escape.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/quote-escape/quote-escape.04.adm
new file mode 100644
index 0000000..2f9e206
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/quote-escape/quote-escape.04.adm
@@ -0,0 +1,4 @@
+{ "id": 1, "name": "Macbook1", "amount": 123.2, "accountNumber": 345.34 }
+{ "id": 2, "name": "Macbook2", "amount": 456.7, "accountNumber": 123.45 }
+{ "id": 3, "name": "Macbook3", "amount": 789.1, "accountNumber": 678.90 }
+{ "id": 4, "name": "Mac|,book4", "amount": 234.5, "accountNumber": 567.89 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/quote-escape/quote-escape.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/quote-escape/quote-escape.13.adm
new file mode 100644
index 0000000..2f9e206
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/quote-escape/quote-escape.13.adm
@@ -0,0 +1,4 @@
+{ "id": 1, "name": "Macbook1", "amount": 123.2, "accountNumber": 345.34 }
+{ "id": 2, "name": "Macbook2", "amount": 456.7, "accountNumber": 123.45 }
+{ "id": 3, "name": "Macbook3", "amount": 789.1, "accountNumber": 678.90 }
+{ "id": 4, "name": "Mac|,book4", "amount": 234.5, "accountNumber": 567.89 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/simple-csv/simple-csv.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/simple-csv/simple-csv.05.adm
new file mode 100644
index 0000000..90e44c4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/simple-csv/simple-csv.05.adm
@@ -0,0 +1,11 @@
+{ "id": 1, "name": "IamNull", "amount": 123.2, "accountNumber": 345.34 }
+{ "id": 2, "name": "IamNull", "amount": 456.7, "accountNumber": 123.45 }
+{ "id": 3, "name": "IamNull", "amount": 789.1, "accountNumber": 678.90 }
+{ "id": 4, "name": "IamNull", "amount": 234.5, "accountNumber": 567.89 }
+{ "id": 5, "name": "IamNull", "amount": 876.5, "accountNumber": 345.67 }
+{ "id": 6, "name": "IamNull", "amount": 345.6, "accountNumber": 987.65 }
+{ "id": 7, "name": "IamNull", "amount": 678.9, "accountNumber": 234.56 }
+{ "id": 8, "name": "IamNull", "amount": 987.2, "accountNumber": 789.12 }
+{ "id": 9, "name": "IamNull", "amount": 543.2, "accountNumber": 321.45 }
+{ "id": 10, "name": "IamNull", "amount": 123.9, "accountNumber": 654.32 }
+{ "id": 11, "name": "IamNull", "amount": 567.8, "accountNumber": 456.78 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/simple-csv/simple-csv.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/simple-csv/simple-csv.13.adm
new file mode 100644
index 0000000..2c305f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/simple-csv/simple-csv.13.adm
@@ -0,0 +1,11 @@
+{ "id": 1, "name": "IamNull", "amount": 123.2, "accountNumber": 345.34 }
+{ "id": 2, "name": "Macbook2", "amount": 456.7, "accountNumber": 123.45 }
+{ "id": 3, "name": "Macbook3", "amount": 789.1, "accountNumber": 678.90 }
+{ "id": 4, "name": "Macbook4", "amount": 234.5, "accountNumber": 567.89 }
+{ "id": 5, "name": "Macbook5", "amount": 876.5, "accountNumber": 345.67 }
+{ "id": 6, "name": "Macbook6", "amount": 345.6, "accountNumber": 987.65 }
+{ "id": 7, "name": "Macbook7", "amount": 678.9, "accountNumber": 234.56 }
+{ "id": 8, "name": "Macbook8", "amount": 987.2, "accountNumber": 789.12 }
+{ "id": 9, "name": "Macbook9", "amount": 543.2, "accountNumber": 321.45 }
+{ "id": 10, "name": "Macbook10", "amount": 123.9, "accountNumber": 654.32 }
+{ "id": 11, "name": "Macbook11", "amount": 567.8, "accountNumber": 456.78 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/type-mismatch/type-mismatch.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/type-mismatch/type-mismatch.04.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/csv/type-mismatch/type-mismatch.04.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/parquet-cover-data-types/parquet-cover-data-types.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/parquet-cover-data-types/parquet-cover-data-types.05.adm
new file mode 100644
index 0000000..8fc863e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/parquet-cover-data-types/parquet-cover-data-types.05.adm
@@ -0,0 +1 @@
+{ "name": "Virat", "id": 18, "dateType": date("1988-11-05"), "timeType": time("03:10:00.493"), "boolType": false, "doubleType": 0.75, "datetimeType": datetime("1900-02-01T00:00:00.000") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/parquet-empty-array/parquet-empty-array.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/parquet-empty-array/parquet-empty-array.05.adm
new file mode 100644
index 0000000..97de7e9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/parquet-empty-array/parquet-empty-array.05.adm
@@ -0,0 +1,2 @@
+{ "id": 2, "name": { "first": [ "power", "star" ] } }
+{ "id": 5, "name": { "first": [  ] } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/parquet-heterogeneous/parquet-heterogeneous.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/parquet-heterogeneous/parquet-heterogeneous.05.adm
new file mode 100644
index 0000000..4ecacdb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/parquet-heterogeneous/parquet-heterogeneous.05.adm
@@ -0,0 +1,11 @@
+{ "id": 2 }
+{ "name": "virat", "id": 5 }
+{ "name": { "first": "virat" }, "id": 8 }
+{ "name": { "first": "virat" }, "id": 10, "age": 18 }
+{ "name": 123, "id": 12 }
+{ "name": [ 123, 456 ], "id": 15 }
+{ "name": 765, "id": 17 }
+{ "name": [ 789 ], "id": 20 }
+{ "name": [ { "first": "virat" } ], "id": 21 }
+{ "name": [ { "first": "virat", "second": "kohli" } ], "id": 27 }
+{ "name": { "first": "virat" }, "id": 28 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.05.adm
new file mode 100644
index 0000000..6b73640
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/parquet-partition-heterogeneous/parquet-partition-heterogeneous.05.adm
@@ -0,0 +1,100 @@
+{ "partitioner_key": "A", "price": 100, "name": { "nickname": "VK" }, "active": [ true, false ], "id": 1, "randomField": 5678 }
+{ "partitioner_key": "A", "price": { "amount": 99.99, "currency": "USD" }, "name": [ "Kohli", "Dhoni" ], "active": 1, "id": 2, "randomField": 18 }
+{ "partitioner_key": "C", "price": "one hundred", "name": false, "active": { "status": "active" }, "id": 3, "randomField": [ 1, 2, 3 ] }
+{ "partitioner_key": "C", "price": 100, "name": { "nickname": "VK" }, "active": [ true, false ], "id": 4, "randomField": 5678 }
+{ "partitioner_key": "A", "price": "99.99", "name": { "last": "Kohli", "first": "Virat" }, "active": "yes", "id": 5, "randomField": "1234" }
+{ "partitioner_key": "C", "price": "101.50", "name": 3456, "active": true, "id": 6, "randomField": { "value": 7890 } }
+{ "partitioner_key": "A", "price": { "amount": 99.99, "currency": "USD" }, "name": [ "Kohli", "Dhoni" ], "active": 1, "id": 7, "randomField": 18 }
+{ "partitioner_key": "B", "price": "one hundred", "name": false, "active": { "status": "active" }, "id": 8, "randomField": [ 1, 2, 3 ] }
+{ "partitioner_key": "A", "price": { "amount": 99.99, "currency": "USD" }, "name": [ "Kohli", "Dhoni" ], "active": 1, "id": 9, "randomField": 18 }
+{ "partitioner_key": "B", "price": "101.50", "name": 3456, "active": true, "id": 10, "randomField": { "value": 7890 } }
+{ "partitioner_key": "B", "price": "101.50", "name": 3456, "active": true, "id": 11, "randomField": { "value": 7890 } }
+{ "partitioner_key": "B", "price": { "currency": "INR", "value": 99.99 }, "name": "None", "active": false, "id": 12, "randomField": "5678" }
+{ "partitioner_key": "A", "price": 50, "name": [ "Dhoni", "Kohli" ], "active": 0, "id": 13, "randomField": "None" }
+{ "partitioner_key": "A", "price": 100, "name": { "nickname": "VK" }, "active": [ true, false ], "id": 14, "randomField": 5678 }
+{ "partitioner_key": "C", "price": { "amount": 99.99, "currency": "USD" }, "name": [ "Kohli", "Dhoni" ], "active": 1, "id": 15, "randomField": 18 }
+{ "partitioner_key": "C", "price": [ 99, 100, 101 ], "name": 9876, "active": "active", "id": 16, "randomField": { "identifier": "ORD1234", "type": "order" } }
+{ "partitioner_key": "B", "price": [ 99, 100, 101 ], "name": 9876, "active": "active", "id": 17, "randomField": { "identifier": "ORD1234", "type": "order" } }
+{ "partitioner_key": "C", "price": 100, "name": { "nickname": "VK" }, "active": [ true, false ], "id": 18, "randomField": 5678 }
+{ "partitioner_key": "C", "price": [ 99, 100, 101 ], "name": 9876, "active": "active", "id": 19, "randomField": { "identifier": "ORD1234", "type": "order" } }
+{ "partitioner_key": "B", "price": 101.5, "name": { "last": "Dhoni", "first": "MS" }, "active": "no", "id": 20, "randomField": [ 9, 8, 7 ] }
+{ "partitioner_key": "B", "price": "99.99", "name": { "last": "Kohli", "first": "Virat" }, "active": "yes", "id": 21, "randomField": "1234" }
+{ "partitioner_key": "A", "price": "one hundred", "name": false, "active": { "status": "active" }, "id": 22, "randomField": [ 1, 2, 3 ] }
+{ "partitioner_key": "C", "price": { "amount": 99.99, "currency": "USD" }, "name": [ "Kohli", "Dhoni" ], "active": 1, "id": 23, "randomField": 18 }
+{ "partitioner_key": "B", "price": "one hundred", "name": false, "active": { "status": "active" }, "id": 24, "randomField": [ 1, 2, 3 ] }
+{ "partitioner_key": "C", "price": 50, "name": [ "Dhoni", "Kohli" ], "active": 0, "id": 25, "randomField": "None" }
+{ "partitioner_key": "A", "price": "one hundred", "name": false, "active": { "status": "active" }, "id": 26, "randomField": [ 1, 2, 3 ] }
+{ "partitioner_key": "A", "price": { "currency": "INR", "value": 99.99 }, "name": "None", "active": false, "id": 27, "randomField": "5678" }
+{ "partitioner_key": "B", "price": 101.5, "name": { "last": "Dhoni", "first": "MS" }, "active": "no", "id": 28, "randomField": [ 9, 8, 7 ] }
+{ "partitioner_key": "C", "price": "99.99", "name": { "last": "Kohli", "first": "Virat" }, "active": "yes", "id": 29, "randomField": "1234" }
+{ "partitioner_key": "C", "price": [ 99, 100, 101 ], "name": 9876, "active": "active", "id": 30, "randomField": { "identifier": "ORD1234", "type": "order" } }
+{ "partitioner_key": "C", "price": { "currency": "INR", "value": 99.99 }, "name": "None", "active": false, "id": 31, "randomField": "5678" }
+{ "partitioner_key": "B", "price": 100, "name": { "nickname": "VK" }, "active": [ true, false ], "id": 32, "randomField": 5678 }
+{ "partitioner_key": "C", "price": "101.50", "name": 3456, "active": true, "id": 33, "randomField": { "value": 7890 } }
+{ "partitioner_key": "B", "price": "101.50", "name": 3456, "active": true, "id": 34, "randomField": { "value": 7890 } }
+{ "partitioner_key": "B", "price": [ 99, 100, 101 ], "name": 9876, "active": "active", "id": 35, "randomField": { "identifier": "ORD1234", "type": "order" } }
+{ "partitioner_key": "B", "price": 50, "name": [ "Dhoni", "Kohli" ], "active": 0, "id": 36, "randomField": "None" }
+{ "partitioner_key": "B", "price": "101.50", "name": 3456, "active": true, "id": 37, "randomField": { "value": 7890 } }
+{ "partitioner_key": "A", "price": [ 99, 100, 101 ], "name": 9876, "active": "active", "id": 38, "randomField": { "identifier": "ORD1234", "type": "order" } }
+{ "partitioner_key": "B", "price": 100, "name": { "nickname": "VK" }, "active": [ true, false ], "id": 39, "randomField": 5678 }
+{ "partitioner_key": "C", "price": "99.99", "name": { "last": "Kohli", "first": "Virat" }, "active": "yes", "id": 40, "randomField": "1234" }
+{ "partitioner_key": "A", "price": "one hundred", "name": false, "active": { "status": "active" }, "id": 41, "randomField": [ 1, 2, 3 ] }
+{ "partitioner_key": "A", "price": { "currency": "INR", "value": 99.99 }, "name": "None", "active": false, "id": 42, "randomField": "5678" }
+{ "partitioner_key": "C", "price": 101.5, "name": { "last": "Dhoni", "first": "MS" }, "active": "no", "id": 43, "randomField": [ 9, 8, 7 ] }
+{ "partitioner_key": "C", "price": { "amount": 99.99, "currency": "USD" }, "name": [ "Kohli", "Dhoni" ], "active": 1, "id": 44, "randomField": 18 }
+{ "partitioner_key": "A", "price": 99.99, "name": "Virat Kohli", "active": true, "id": 45, "randomField": 1234 }
+{ "partitioner_key": "B", "price": "one hundred", "name": false, "active": { "status": "active" }, "id": 46, "randomField": [ 1, 2, 3 ] }
+{ "partitioner_key": "C", "price": 50, "name": [ "Dhoni", "Kohli" ], "active": 0, "id": 47, "randomField": "None" }
+{ "partitioner_key": "A", "price": 50, "name": [ "Dhoni", "Kohli" ], "active": 0, "id": 48, "randomField": "None" }
+{ "partitioner_key": "A", "price": 101.5, "name": { "last": "Dhoni", "first": "MS" }, "active": "no", "id": 49, "randomField": [ 9, 8, 7 ] }
+{ "partitioner_key": "C", "price": 101.5, "name": { "last": "Dhoni", "first": "MS" }, "active": "no", "id": 50, "randomField": [ 9, 8, 7 ] }
+{ "partitioner_key": "B", "price": 50, "name": [ "Dhoni", "Kohli" ], "active": 0, "id": 51, "randomField": "None" }
+{ "partitioner_key": "A", "price": 50, "name": [ "Dhoni", "Kohli" ], "active": 0, "id": 52, "randomField": "None" }
+{ "partitioner_key": "C", "price": "101.50", "name": 3456, "active": true, "id": 53, "randomField": { "value": 7890 } }
+{ "partitioner_key": "B", "price": 101.5, "name": { "last": "Dhoni", "first": "MS" }, "active": "no", "id": 54, "randomField": [ 9, 8, 7 ] }
+{ "partitioner_key": "A", "price": { "currency": "INR", "value": 99.99 }, "name": "None", "active": false, "id": 55, "randomField": "5678" }
+{ "partitioner_key": "B", "price": 99.99, "name": "Virat Kohli", "active": true, "id": 56, "randomField": 1234 }
+{ "partitioner_key": "B", "price": "one hundred", "name": false, "active": { "status": "active" }, "id": 57, "randomField": [ 1, 2, 3 ] }
+{ "partitioner_key": "B", "price": "101.50", "name": 3456, "active": true, "id": 58, "randomField": { "value": 7890 } }
+{ "partitioner_key": "A", "price": 50, "name": [ "Dhoni", "Kohli" ], "active": 0, "id": 59, "randomField": "None" }
+{ "partitioner_key": "C", "price": { "amount": 99.99, "currency": "USD" }, "name": [ "Kohli", "Dhoni" ], "active": 1, "id": 60, "randomField": 18 }
+{ "partitioner_key": "C", "price": [ 99, 100, 101 ], "name": 9876, "active": "active", "id": 61, "randomField": { "identifier": "ORD1234", "type": "order" } }
+{ "partitioner_key": "A", "price": 101.5, "name": { "last": "Dhoni", "first": "MS" }, "active": "no", "id": 62, "randomField": [ 9, 8, 7 ] }
+{ "partitioner_key": "A", "price": "one hundred", "name": false, "active": { "status": "active" }, "id": 63, "randomField": [ 1, 2, 3 ] }
+{ "partitioner_key": "C", "price": { "amount": 99.99, "currency": "USD" }, "name": [ "Kohli", "Dhoni" ], "active": 1, "id": 64, "randomField": 18 }
+{ "partitioner_key": "A", "price": 100, "name": { "nickname": "VK" }, "active": [ true, false ], "id": 65, "randomField": 5678 }
+{ "partitioner_key": "B", "price": 101.5, "name": { "last": "Dhoni", "first": "MS" }, "active": "no", "id": 66, "randomField": [ 9, 8, 7 ] }
+{ "partitioner_key": "B", "price": { "currency": "INR", "value": 99.99 }, "name": "None", "active": false, "id": 67, "randomField": "5678" }
+{ "partitioner_key": "C", "price": "101.50", "name": 3456, "active": true, "id": 68, "randomField": { "value": 7890 } }
+{ "partitioner_key": "C", "price": 50, "name": [ "Dhoni", "Kohli" ], "active": 0, "id": 69, "randomField": "None" }
+{ "partitioner_key": "A", "price": { "amount": 99.99, "currency": "USD" }, "name": [ "Kohli", "Dhoni" ], "active": 1, "id": 70, "randomField": 18 }
+{ "partitioner_key": "A", "price": { "amount": 99.99, "currency": "USD" }, "name": [ "Kohli", "Dhoni" ], "active": 1, "id": 71, "randomField": 18 }
+{ "partitioner_key": "B", "price": { "amount": 99.99, "currency": "USD" }, "name": [ "Kohli", "Dhoni" ], "active": 1, "id": 72, "randomField": 18 }
+{ "partitioner_key": "C", "price": "one hundred", "name": false, "active": { "status": "active" }, "id": 73, "randomField": [ 1, 2, 3 ] }
+{ "partitioner_key": "C", "price": 99.99, "name": "Virat Kohli", "active": true, "id": 74, "randomField": 1234 }
+{ "partitioner_key": "A", "price": "101.50", "name": 3456, "active": true, "id": 75, "randomField": { "value": 7890 } }
+{ "partitioner_key": "B", "price": 50, "name": [ "Dhoni", "Kohli" ], "active": 0, "id": 76, "randomField": "None" }
+{ "partitioner_key": "A", "price": "one hundred", "name": false, "active": { "status": "active" }, "id": 77, "randomField": [ 1, 2, 3 ] }
+{ "partitioner_key": "A", "price": 101.5, "name": { "last": "Dhoni", "first": "MS" }, "active": "no", "id": 78, "randomField": [ 9, 8, 7 ] }
+{ "partitioner_key": "C", "price": "99.99", "name": { "last": "Kohli", "first": "Virat" }, "active": "yes", "id": 79, "randomField": "1234" }
+{ "partitioner_key": "A", "price": "99.99", "name": { "last": "Kohli", "first": "Virat" }, "active": "yes", "id": 80, "randomField": "1234" }
+{ "partitioner_key": "C", "price": [ 99, 100, 101 ], "name": 9876, "active": "active", "id": 81, "randomField": { "identifier": "ORD1234", "type": "order" } }
+{ "partitioner_key": "A", "price": 101.5, "name": { "last": "Dhoni", "first": "MS" }, "active": "no", "id": 82, "randomField": [ 9, 8, 7 ] }
+{ "partitioner_key": "A", "price": { "currency": "INR", "value": 99.99 }, "name": "None", "active": false, "id": 83, "randomField": "5678" }
+{ "partitioner_key": "A", "price": 101.5, "name": { "last": "Dhoni", "first": "MS" }, "active": "no", "id": 84, "randomField": [ 9, 8, 7 ] }
+{ "partitioner_key": "C", "price": 50, "name": [ "Dhoni", "Kohli" ], "active": 0, "id": 85, "randomField": "None" }
+{ "partitioner_key": "C", "price": "99.99", "name": { "last": "Kohli", "first": "Virat" }, "active": "yes", "id": 86, "randomField": "1234" }
+{ "partitioner_key": "C", "price": { "amount": 99.99, "currency": "USD" }, "name": [ "Kohli", "Dhoni" ], "active": 1, "id": 87, "randomField": 18 }
+{ "partitioner_key": "A", "price": 100, "name": { "nickname": "VK" }, "active": [ true, false ], "id": 88, "randomField": 5678 }
+{ "partitioner_key": "C", "price": "99.99", "name": { "last": "Kohli", "first": "Virat" }, "active": "yes", "id": 89, "randomField": "1234" }
+{ "partitioner_key": "C", "price": 50, "name": [ "Dhoni", "Kohli" ], "active": 0, "id": 90, "randomField": "None" }
+{ "partitioner_key": "A", "price": 99.99, "name": "Virat Kohli", "active": true, "id": 91, "randomField": 1234 }
+{ "partitioner_key": "A", "price": [ 99, 100, 101 ], "name": 9876, "active": "active", "id": 92, "randomField": { "identifier": "ORD1234", "type": "order" } }
+{ "partitioner_key": "B", "price": "101.50", "name": 3456, "active": true, "id": 93, "randomField": { "value": 7890 } }
+{ "partitioner_key": "A", "price": 101.5, "name": { "last": "Dhoni", "first": "MS" }, "active": "no", "id": 94, "randomField": [ 9, 8, 7 ] }
+{ "partitioner_key": "B", "price": 50, "name": [ "Dhoni", "Kohli" ], "active": 0, "id": 95, "randomField": "None" }
+{ "partitioner_key": "B", "price": 99.99, "name": "Virat Kohli", "active": true, "id": 96, "randomField": 1234 }
+{ "partitioner_key": "A", "price": [ 99, 100, 101 ], "name": 9876, "active": "active", "id": 97, "randomField": { "identifier": "ORD1234", "type": "order" } }
+{ "partitioner_key": "B", "price": "99.99", "name": { "last": "Kohli", "first": "Virat" }, "active": "yes", "id": 98, "randomField": "1234" }
+{ "partitioner_key": "A", "price": [ 99, 100, 101 ], "name": 9876, "active": "active", "id": 99, "randomField": { "identifier": "ORD1234", "type": "order" } }
+{ "partitioner_key": "C", "price": "99.99", "name": { "last": "Kohli", "first": "Virat" }, "active": "yes", "id": 100, "randomField": "1234" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/parquet-simple/parquet-simple.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/parquet-simple/parquet-simple.04.adm
new file mode 100644
index 0000000..bf567b2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/parquet-simple/parquet-simple.04.adm
@@ -0,0 +1 @@
+{ "id": "123" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/parquet-tweet/parquet-tweet.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/parquet-tweet/parquet-tweet.05.adm
new file mode 100644
index 0000000..5e0df96
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/parquet-tweet/parquet-tweet.05.adm
@@ -0,0 +1,2 @@
+{ "coordinates": { "coordinates": [ 1.1 ], "type": "string" }, "created_at": "string", "entities": { "urls": [ { "display_url": "string", "expanded_url": "string", "indices": [ 1 ], "url": "string" } ], "user_mentions": [ { "id": 1, "id_str": "string", "indices": [ 1 ], "name": "string", "screen_name": "string" } ] }, "favorite_count": 1, "favorited": true, "filter_level": "string", "geo": { "coordinates": [ 1.1 ], "type": "string" }, "id": "0000000", "id_str": "string", "in_reply_to_screen_name": "string", "in_reply_to_status_id": 1, "in_reply_to_status_id_str": "string", "in_reply_to_user_id": 1, "in_reply_to_user_id_str": "string", "is_quote_status": true, "lang": "string", "place": { "bounding_box": { "coordinates": [ [ [ 1.1 ] ] ], "type": "string" }, "country": "string", "country_code": "string", "full_name": "string", "id": "string", "name": "string", "place_type": "string", "url": "string" }, "possibly_sensitive": true, "quoted_status": { "created_at": "string", "entities": { "user_mentions": [ { "id": 1, "id_str": "string", "indices": [ 1 ], "name": "string", "screen_name": "string" } ] }, "favorite_count": 1, "favorited": true, "filter_level": "string", "id": 1, "id_str": "string", "in_reply_to_screen_name": "string", "in_reply_to_status_id": 1, "in_reply_to_status_id_str": "string", "in_reply_to_user_id": 1, "in_reply_to_user_id_str": "string", "is_quote_status": true, "lang": "string", "retweet_count": 1, "retweeted": true, "source": "string", "text": "string", "truncated": true, "user": { "contributors_enabled": true, "created_at": "string", "default_profile": true, "default_profile_image": true, "description": "string", "favourites_count": 1, "followers_count": 1, "friends_count": 1, "geo_enabled": true, "id": 1, "id_str": "string", "is_translator": true, "lang": "string", "listed_count": 1, "name": "string", "profile_background_color": "string", "profile_background_image_url": "string", "profile_background_image_url_https": "string", "profile_background_tile": true, "profile_banner_url": "string", "profile_image_url": "string", "profile_image_url_https": "string", "profile_link_color": "string", "profile_sidebar_border_color": "string", "profile_sidebar_fill_color": "string", "profile_text_color": "string", "profile_use_background_image": true, "protected": true, "screen_name": "string", "statuses_count": 1, "verified": true } }, "quoted_status_id": 1, "quoted_status_id_str": "string", "retweet_count": 1, "retweeted": true, "source": "string", "text": "string", "timestamp_ms": "string", "truncated": true, "user": { "contributors_enabled": true, "created_at": "string", "default_profile": true, "default_profile_image": true, "description": "string", "favourites_count": 1, "followers_count": 1, "friends_count": 1, "geo_enabled": true, "id": 1, "id_str": "string", "is_translator": true, "lang": "string", "listed_count": 1, "location": "string", "name": "string", "profile_background_color": "string", "profile_background_image_url": "string", "profile_background_image_url_https": "string", "profile_background_tile": true, "profile_banner_url": "string", "profile_image_url": "string", "profile_image_url_https": "string", "profile_link_color": "string", "profile_sidebar_border_color": "string", "profile_sidebar_fill_color": "string", "profile_text_color": "string", "profile_use_background_image": true, "protected": true, "screen_name": "string", "statuses_count": 1, "time_zone": "string", "url": "string", "utc_offset": 1, "verified": true } }
+{ "coordinates": { "coordinates": [ 1.1 ], "type": "string" }, "created_at": "string", "favorite_count": 1, "favorited": true, "filter_level": "string", "geo": { "coordinates": [ 1.1 ], "type": "string" }, "id": "11111111111111111111", "id_str": "string", "in_reply_to_screen_name": "string", "in_reply_to_status_id": 1, "in_reply_to_status_id_str": "string", "in_reply_to_user_id": 1, "in_reply_to_user_id_str": "string", "is_quote_status": true, "lang": "string", "place": { "bounding_box": { "coordinates": [ [ [ 1.1 ] ] ], "type": "string" }, "country": "string", "country_code": "string", "full_name": "string", "id": "string", "name": "string", "place_type": "string", "url": "string" }, "possibly_sensitive": true, "quoted_status": { "created_at": "string", "entities": { "user_mentions": [ { "id": 1, "id_str": "string", "indices": [ 1 ], "name": "string", "screen_name": "string" } ] }, "favorite_count": 1, "favorited": true, "filter_level": "string", "id": 1, "id_str": "string", "in_reply_to_screen_name": "string", "in_reply_to_status_id": 1, "in_reply_to_status_id_str": "string", "in_reply_to_user_id": 1, "in_reply_to_user_id_str": "string", "is_quote_status": true, "lang": "string", "retweet_count": 1, "retweeted": true, "source": "string", "text": "string", "truncated": true, "user": { "contributors_enabled": true, "created_at": "string", "default_profile": true, "default_profile_image": true, "description": "string", "favourites_count": 1, "followers_count": 1, "friends_count": 1, "geo_enabled": true, "id": 1, "id_str": "string", "is_translator": true, "lang": "string", "listed_count": 1, "name": "string", "profile_background_color": "string", "profile_background_image_url": "string", "profile_background_image_url_https": "string", "profile_background_tile": true, "profile_banner_url": "string", "profile_image_url": "string", "profile_image_url_https": "string", "profile_link_color": "string", "profile_sidebar_border_color": "string", "profile_sidebar_fill_color": "string", "profile_text_color": "string", "profile_use_background_image": true, "protected": true, "screen_name": "string", "statuses_count": 1, "verified": true } }, "quoted_status_id": 1, "quoted_status_id_str": "string", "retweet_count": 1, "retweeted": true, "source": "string", "text": "string", "timestamp_ms": "string", "truncated": true, "user": { "contributors_enabled": true, "created_at": "string", "default_profile": true, "default_profile_image": true, "description": "string", "favourites_count": 1, "followers_count": 1, "friends_count": 1, "geo_enabled": true, "id": 1, "id_str": "string", "is_translator": true, "lang": "string", "listed_count": 1, "location": "string", "name": "string", "profile_background_color": "string", "profile_background_image_url": "string", "profile_background_image_url_https": "string", "profile_background_tile": true, "profile_banner_url": "string", "profile_image_url": "string", "profile_image_url_https": "string", "profile_link_color": "string", "profile_sidebar_border_color": "string", "profile_sidebar_fill_color": "string", "profile_text_color": "string", "profile_use_background_image": true, "protected": true, "screen_name": "string", "statuses_count": 1, "time_zone": "string", "url": "string", "utc_offset": 1, "verified": true } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/parquet-utf8/parquet-utf8.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/parquet-utf8/parquet-utf8.05.adm
new file mode 100644
index 0000000..c60145d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/copy-to/parquet-utf8/parquet-utf8.05.adm
@@ -0,0 +1,8 @@
+{ "id": 1, "name": "John" }
+{ "id": 2, "name": "Abel" }
+{ "id": 3, "name": "Sandy" }
+{ "id": 4, "name": "Alex" }
+{ "id": 5, "name": "Mike" }
+{ "id": 6, "name": "Tom" }
+{ "comment": "😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا", "id": 7, "name": "Jerry" }
+{ "comment": "😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا", "id": 8, "name": "William" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.plan
index fb83a56..b44bf46 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.plan
@@ -2,41 +2,37 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$50]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$50] <- [{"deptId": $#1, "star_cost": $$53}] [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]
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          group by ([$#1 := $$58]) decor ([]) {
-                    aggregate [$$53] <- [agg-global-sql-sum($$57)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$58]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-              group by ([$$58 := $$51]) decor ([]) {
-                        aggregate [$$57] <- [agg-local-sql-sum($$48)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$51]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$48, $$51]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$50] <- [{"deptId": $#1, "star_cost": $$53}] project: [$$50] [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]
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        group by ([$#1 := $$58]) decor ([]) {
+                  aggregate [$$53] <- [agg-global-sql-sum($$57)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$58]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
+            group by ([$$58 := $$51]) decor ([]) {
+                      aggregate [$$57] <- [agg-local-sql-sum($$48)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$51]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                assign [$$51, $$48] <- [substring($$e.getField("dept").getField("department_id"), 0), $$e.getField("salary")] project: [$$48, $$51] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$e]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$51, $$48] <- [substring($$e.getField("dept").getField("department_id"), 0), $$e.getField("salary")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
-                      project ([$$e]) [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 []<-[$$52, $$e] <- gby.Employee [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|
-                          data-scan []<-[$$52, $$e] <- gby.Employee [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|
+                          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/explain/explain_field_access_closed/explain_field_access_closed.1.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.plan
index bd96df9..d8a5148 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.plan
@@ -2,41 +2,37 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$49]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$49] <- [{"deptId": $#1, "star_cost": $$52}] [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]
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          group by ([$#1 := $$56]) decor ([]) {
-                    aggregate [$$52] <- [agg-global-sql-sum($$55)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$56]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
-              group by ([$$56 := $$50]) decor ([]) {
-                        aggregate [$$55] <- [agg-local-sql-sum($$47)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$50]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$47, $$50]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$49] <- [{"deptId": $#1, "star_cost": $$52}] project: [$$49] [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]
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        group by ([$#1 := $$56]) decor ([]) {
+                  aggregate [$$52] <- [agg-global-sql-sum($$55)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$56]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
+            group by ([$$56 := $$50]) decor ([]) {
+                      aggregate [$$55] <- [agg-local-sql-sum($$47)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$50]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                assign [$$50, $$47] <- [substring($$e.getField(1), 0), $$e.getField(2)] project: [$$47, $$50] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$e]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$50, $$47] <- [substring($$e.getField(1), 0), $$e.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
-                      project ([$$e]) [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 []<-[$$51, $$e] <- gby.Employee [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|
-                          data-scan []<-[$$51, $$e] <- gby.Employee [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|
+                          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/explain/explain_same_datasource_function_different_arguments/test.030.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_same_datasource_function_different_arguments/test.030.plan
index 825f43a..7b24434 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_same_datasource_function_different_arguments/test.030.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_same_datasource_function_different_arguments/test.030.plan
@@ -6,33 +6,29 @@
     -- UNION_ALL  |UNPARTITIONED|
       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-        project ([$$16]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- STREAM_PROJECT  |UNPARTITIONED|
-          assign [$$16] <- [cast({"intIndex": $$13})] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- ASSIGN  |UNPARTITIONED|
-            aggregate [$$13] <- [listify($$12)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |UNPARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                data-scan []<-[$$12] <- asterix.dump-index. [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|
+        assign [$$16] <- [cast({"intIndex": $$13})] project: [$$16] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- ASSIGN  |UNPARTITIONED|
+          aggregate [$$13] <- [listify($$12)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |UNPARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$12] <- asterix.dump-index. [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|
-        project ([$$17]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- STREAM_PROJECT  |UNPARTITIONED|
-          assign [$$17] <- [cast({"stringIndex": $$15})] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- ASSIGN  |UNPARTITIONED|
-            aggregate [$$15] <- [listify($$14)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |UNPARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                data-scan []<-[$$14] <- asterix.dump-index. [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|
\ No newline at end of file
+        assign [$$17] <- [cast({"stringIndex": $$15})] project: [$$17] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- ASSIGN  |UNPARTITIONED|
+          aggregate [$$15] <- [listify($$14)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |UNPARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$14] <- asterix.dump-index. [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/external-dataset/common/deltalake-field-access-pushdown/deltalake-field-access-pushdown.06.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/deltalake-field-access-pushdown/deltalake-field-access-pushdown.06.plan
index 848bc04..5b67dfe 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/deltalake-field-access-pushdown/deltalake-field-access-pushdown.06.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/deltalake-field-access-pushdown/deltalake-field-access-pushdown.06.plan
@@ -2,49 +2,43 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$32]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$32] <- [{"integer_type": $$34, "string_type": $$37, "timestamp_type": $$38}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$34, $$37, $$38]) [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 [$$36(ASC) ]  |PARTITIONED|
-            order (ASC, $$36) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$34, $$37, $$38, $$36]) [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|
-                    join (eq($$34, $$35)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HYBRID_HASH_JOIN [$$34][$$35]  |PARTITIONED|
-                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
-                        project ([$$34]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$34] <- [$$d1.getField("integer_type")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- ASSIGN  |PARTITIONED|
+    assign [$$32] <- [{"integer_type": $$34, "string_type": $$37, "timestamp_type": $$38}] project: [$$32] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$34, $$37, $$38]) [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 [$$36(ASC) ]  |PARTITIONED|
+          order (ASC, $$36) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              project ([$$34, $$37, $$38, $$36]) [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|
+                  join (eq($$34, $$35)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HYBRID_HASH_JOIN [$$34][$$35]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
+                      assign [$$34] <- [$$d1.getField("integer_type")] project: [$$34] [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]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$d1] <- test.DeltalakeDataset1 [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|
-                              data-scan []<-[$$d1] <- test.DeltalakeDataset1 [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 [$$35]  |PARTITIONED|
-                        project ([$$37, $$38, $$36, $$35]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$38, $$37, $$36, $$35] <- [$$d2.getField("timestamp_type"), $$d2.getField("string_type"), $$d2.getField("id"), $$d2.getField("integer_type")] [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|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
+                      assign [$$38, $$37, $$36, $$35] <- [$$d2.getField("timestamp_type"), $$d2.getField("string_type"), $$d2.getField("id"), $$d2.getField("integer_type")] project: [$$37, $$38, $$36, $$35] [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]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$d2] <- test.DeltalakeDataset2 [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|
-                              data-scan []<-[$$d2] <- test.DeltalakeDataset2 [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|
+                              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/external-dataset/common/deltalake-field-access-pushdown/deltalake-field-access-pushdown.08.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/deltalake-field-access-pushdown/deltalake-field-access-pushdown.08.plan
index 34671dd..226a67c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/deltalake-field-access-pushdown/deltalake-field-access-pushdown.08.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/deltalake-field-access-pushdown/deltalake-field-access-pushdown.08.plan
@@ -2,49 +2,43 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$32]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$32] <- [{"integer_type": $$34, "string_type": $$37, "timestamp_type": $$38}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$34, $$37, $$38]) [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 [$$36(ASC) ]  |PARTITIONED|
-            order (ASC, $$36) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$34, $$37, $$38, $$36]) [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|
-                    join (eq($$34, $$35)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HYBRID_HASH_JOIN [$$34][$$35]  |PARTITIONED|
-                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
-                        project ([$$34]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$34] <- [$$d1.getField("integer_type")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- ASSIGN  |PARTITIONED|
+    assign [$$32] <- [{"integer_type": $$34, "string_type": $$37, "timestamp_type": $$38}] project: [$$32] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$34, $$37, $$38]) [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 [$$36(ASC) ]  |PARTITIONED|
+          order (ASC, $$36) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              project ([$$34, $$37, $$38, $$36]) [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|
+                  join (eq($$34, $$35)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HYBRID_HASH_JOIN [$$34][$$35]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
+                      assign [$$34] <- [$$d1.getField("integer_type")] project: [$$34] [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]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$d1] <- test.DeltalakeDataset1 project ({integer_type:any}) [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|
-                              data-scan []<-[$$d1] <- test.DeltalakeDataset1 project ({integer_type:any}) [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 [$$35]  |PARTITIONED|
-                        project ([$$37, $$38, $$36, $$35]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$38, $$37, $$36, $$35] <- [$$d2.getField("timestamp_type"), $$d2.getField("string_type"), $$d2.getField("id"), $$d2.getField("integer_type")] [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|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
+                      assign [$$38, $$37, $$36, $$35] <- [$$d2.getField("timestamp_type"), $$d2.getField("string_type"), $$d2.getField("id"), $$d2.getField("integer_type")] project: [$$37, $$38, $$36, $$35] [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]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$d2] <- test.DeltalakeDataset2 project ({string_type:any,timestamp_type:any,id:any,integer_type:any}) [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|
-                              data-scan []<-[$$d2] <- test.DeltalakeDataset2 project ({string_type:any,timestamp_type:any,id:any,integer_type:any}) [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|
+                              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/external-dataset/common/deltalake-field-access-pushdown/deltalake-field-access-pushdown.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/deltalake-field-access-pushdown/deltalake-field-access-pushdown.10.adm
index f706ba2..ba1aa87 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/deltalake-field-access-pushdown/deltalake-field-access-pushdown.10.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/deltalake-field-access-pushdown/deltalake-field-access-pushdown.10.adm
@@ -2,45 +2,39 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$32]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$32] <- [{"integer_type": $$34, "date_type": $$37, "decimal_type": $$38}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$34, $$37, $$38]) [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 [$$35(ASC) ]  |PARTITIONED|
-            order (ASC, $$35) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                join (eq($$34, $$35)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- HYBRID_HASH_JOIN [$$34][$$35]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
-                    project ([$$34, $$37]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$37, $$34] <- [$$d1.getField("date_type"), $$d1.getField("integer_type")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
+    assign [$$32] <- [{"integer_type": $$34, "date_type": $$37, "decimal_type": $$38}] project: [$$32] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$34, $$37, $$38]) [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 [$$35(ASC) ]  |PARTITIONED|
+          order (ASC, $$35) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              join (eq($$34, $$35)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- HYBRID_HASH_JOIN [$$34][$$35]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
+                  assign [$$37, $$34] <- [$$d1.getField("date_type"), $$d1.getField("integer_type")] project: [$$34, $$37] [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]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$d1] <- test.DeltalakeDataset1 project ({date_type:any,integer_type:any}) [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|
-                          data-scan []<-[$$d1] <- test.DeltalakeDataset1 project ({date_type:any,integer_type:any}) [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 [$$35]  |PARTITIONED|
-                    project ([$$38, $$35]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$38, $$35] <- [$$d2.getField("decimal_type"), $$d2.getField("integer_type")] [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|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
+                  assign [$$38, $$35] <- [$$d2.getField("decimal_type"), $$d2.getField("integer_type")] project: [$$38, $$35] [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]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$d2] <- test.DeltalakeDataset2 project ({decimal_type:any,integer_type:any}) [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|
-                          data-scan []<-[$$d2] <- test.DeltalakeDataset2 project ({decimal_type:any,integer_type:any}) [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|
+                          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/external-dataset/common/deltalake-file-nine/deltalake-file-nine.01.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/deltalake-file-nine/deltalake-file-nine.01.adm
new file mode 100644
index 0000000..500f6a9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/deltalake-file-nine/deltalake-file-nine.01.adm
@@ -0,0 +1,9 @@
+{ "id": 0, "name": "Cooper" }
+{ "id": 1, "name": "Adam" }
+{ "id": 2, "name": "Third" }
+{ "id": 3, "name": "Fourth" }
+{ "id": 4, "name": "Five" }
+{ "id": 5, "name": "Six" }
+{ "id": 6, "name": "Seven" }
+{ "id": 7, "name": "Eight" }
+{ "id": 8, "name": "Nine" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/deltalake-file-one/deltalake-file-one.01.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/deltalake-file-one/deltalake-file-one.01.adm
new file mode 100644
index 0000000..006681c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/deltalake-file-one/deltalake-file-one.01.adm
@@ -0,0 +1 @@
+{ "id": 0, "name": "Cooper" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/embed-one-value/one-field.131.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/embed-one-value/one-field.131.plan
index 4edf24e..afef4ab 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/embed-one-value/one-field.131.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/embed-one-value/one-field.131.plan
@@ -10,17 +10,15 @@
         -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$28]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select (and(eq(lowercase($$25.getField("first")), "john"), eq(string-concat(ordered-list-constructor("john ", lowercase($$25.getField("last")))), "john smith"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                assign [$$28, $$25] <- [$$d.getField("id"), $$d.getField("name")] [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]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$d] <- test.LastName prefix-filter on: eq(string-concat(ordered-list-constructor("john ", lowercase($$d.getField("name").getField("last")))), "john smith") embed-filter-value: true [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|
+            select (and(eq(lowercase($$25.getField("first")), "john"), eq(string-concat(ordered-list-constructor("john ", lowercase($$25.getField("last")))), "john smith"))) project: [$$d, $$28] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$28, $$25] <- [$$d.getField("id"), $$d.getField("name")] [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]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$d] <- test.LastName prefix-filter on: eq(string-concat(ordered-list-constructor("john ", lowercase($$d.getField("name").getField("last")))), "john smith") embed-filter-value: true [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/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.073.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.073.plan
index 02693c3..f8b544d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.073.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.073.plan
@@ -10,17 +10,15 @@
         -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$27]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select (or(or(eq($$30, "hr"), eq($$30, "accounting")), eq($$d.getField("name").getField("last"), "Smith"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                assign [$$30, $$27] <- [lowercase($$d.getField("department")), $$d.getField("id")] [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]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$d] <- test.Department [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|
+            select (or(or(eq($$30, "hr"), eq($$30, "accounting")), eq($$d.getField("name").getField("last"), "Smith"))) project: [$$d, $$27] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$30, $$27] <- [lowercase($$d.getField("department")), $$d.getField("id")] [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]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$d] <- test.Department [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/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.077.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.077.plan
index 3a8b00d..2015476 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.077.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.077.plan
@@ -10,17 +10,15 @@
         -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$28]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select (or(eq($$31, "hr"), eq($$31, "accounting"), eq($$d.getField("name").getField("last"), "Smith"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                assign [$$31, $$28] <- [lowercase($$d.getField("department")), $$d.getField("id")] [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]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$d] <- test.Department [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|
+            select (or(eq($$31, "hr"), eq($$31, "accounting"), eq($$d.getField("name").getField("last"), "Smith"))) project: [$$d, $$28] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$31, $$28] <- [lowercase($$d.getField("department")), $$d.getField("id")] [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]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$d] <- test.Department [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/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.131.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.131.plan
index 44fd334..929d06d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.131.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.131.plan
@@ -10,17 +10,15 @@
         -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$28]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select (and(eq(lowercase($$25.getField("first")), "john"), eq(string-concat(ordered-list-constructor("john ", lowercase($$25.getField("last")))), "john smith"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                assign [$$28, $$25] <- [$$d.getField("id"), $$d.getField("name")] [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]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$d] <- test.LastName prefix-filter on: eq(string-concat(ordered-list-constructor("john ", lowercase($$d.getField("name").getField("last")))), "john smith") [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|
+            select (and(eq(lowercase($$25.getField("first")), "john"), eq(string-concat(ordered-list-constructor("john ", lowercase($$25.getField("last")))), "john smith"))) project: [$$d, $$28] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$28, $$25] <- [$$d.getField("id"), $$d.getField("name")] [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]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$d] <- test.LastName prefix-filter on: eq(string-concat(ordered-list-constructor("john ", lowercase($$d.getField("name").getField("last")))), "john smith") [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/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.211.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.211.plan
index 646bb04..fd9f409 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.211.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.211.plan
@@ -26,42 +26,38 @@
                         -- HYBRID_HASH_JOIN [$$39][$$43]  |PARTITIONED|
                           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            project ([$$d, $$45, $$34, $$39]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              assign [$$39, $$34] <- [lowercase($$48), lowercase($$d.getField("department"))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ASSIGN  |PARTITIONED|
-                                project ([$$d, $$45, $$48]) [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|
-                                    join (eq($$45, $$46)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- HYBRID_HASH_JOIN [$$45][$$46]  |PARTITIONED|
-                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- HASH_PARTITION_EXCHANGE [$$45]  |PARTITIONED|
-                                        assign [$$45] <- [$$d.getField("id")] [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]
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            data-scan []<-[$$d] <- test.Department prefix-filter on: or(eq(lowercase($$d.getField("department")), "hr"), eq(lowercase($$d.getField("department")), "accounting")) [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 [$$46]  |PARTITIONED|
-                                        project ([$$48, $$46]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          assign [$$48, $$46] <- [$$l.getField("name").getField("last"), $$l.getField("id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- ASSIGN  |PARTITIONED|
+                            assign [$$39, $$34] <- [lowercase($$48), lowercase($$d.getField("department"))] project: [$$d, $$45, $$34, $$39] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- ASSIGN  |PARTITIONED|
+                              project ([$$d, $$45, $$48]) [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|
+                                  join (eq($$45, $$46)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- HYBRID_HASH_JOIN [$$45][$$46]  |PARTITIONED|
+                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- HASH_PARTITION_EXCHANGE [$$45]  |PARTITIONED|
+                                      assign [$$45] <- [$$d.getField("id")] [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]
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          data-scan []<-[$$d] <- test.Department prefix-filter on: or(eq(lowercase($$d.getField("department")), "hr"), eq(lowercase($$d.getField("department")), "accounting")) [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|
-                                              data-scan []<-[$$l] <- test.LastName prefix-filter on: or(eq(lowercase($$l.getField("name").getField("last")), "miller"), eq(lowercase($$l.getField("name").getField("last")), "smith")) [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|
+                                              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 [$$46]  |PARTITIONED|
+                                      assign [$$48, $$46] <- [$$l.getField("name").getField("last"), $$l.getField("id")] project: [$$48, $$46] [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]
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          data-scan []<-[$$l] <- test.LastName prefix-filter on: or(eq(lowercase($$l.getField("name").getField("last")), "miller"), eq(lowercase($$l.getField("name").getField("last")), "smith")) [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]
                           -- BROADCAST_EXCHANGE  |PARTITIONED|
                             unnest $$43 <- scan-collection(array: [ "miller", "smith" ]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.221.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.221.plan
index 6b83b9e..1c924a2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.221.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.221.plan
@@ -38,20 +38,18 @@
                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                           -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
-                            project ([$$42]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              select (eq(lowercase($$l.getField("name").getField("last")), "smith")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_SELECT  |PARTITIONED|
-                                assign [$$42] <- [$$l.getField("id")] [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]
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    data-scan []<-[$$l] <- test.LastName prefix-filter on: eq(lowercase($$l.getField("name").getField("last")), "smith") [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|
+                            select (eq(lowercase($$l.getField("name").getField("last")), "smith")) project: [$$42] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- STREAM_SELECT  |PARTITIONED|
+                              assign [$$42] <- [$$l.getField("id")] [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]
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  data-scan []<-[$$l] <- test.LastName prefix-filter on: eq(lowercase($$l.getField("name").getField("last")), "smith") [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]
                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                     unnest $$40 <- scan-collection(array: [ "hr", "accounting" ]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.231.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.231.plan
index c6a4c95..ad9c368 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.231.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/one-field/one-field.231.plan
@@ -10,39 +10,35 @@
         -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$42]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select (or(or(eq($$32, "hr"), eq($$32, "accounting")), eq($$41, "smith"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$d, $$32, $$42, $$41]) [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|
-                    join (eq($$42, $$43)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HYBRID_HASH_JOIN [$$42][$$43]  |PARTITIONED|
-                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
-                        assign [$$32, $$42] <- [lowercase($$d.getField("department")), $$d.getField("id")] [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]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            data-scan []<-[$$d] <- test.Department [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 [$$43]  |PARTITIONED|
-                        project ([$$41, $$43]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$41, $$43] <- [lowercase($$l.getField("name").getField("last")), $$l.getField("id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- ASSIGN  |PARTITIONED|
+            select (or(or(eq($$32, "hr"), eq($$32, "accounting")), eq($$41, "smith"))) project: [$$d, $$42] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$d, $$32, $$42, $$41]) [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|
+                  join (eq($$42, $$43)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HYBRID_HASH_JOIN [$$42][$$43]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
+                      assign [$$32, $$42] <- [lowercase($$d.getField("department")), $$d.getField("id")] [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]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$d] <- test.Department [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|
-                              data-scan []<-[$$l] <- test.LastName [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|
+                              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 [$$43]  |PARTITIONED|
+                      assign [$$41, $$43] <- [lowercase($$l.getField("name").getField("last")), $$l.getField("id")] project: [$$41, $$43] [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]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$l] <- test.LastName [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/external-dataset/common/dynamic-prefixes/avro/views/result.002.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/views/result.002.plan
index 2478db2..eaa9ad6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/views/result.002.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/views/result.002.plan
@@ -2,21 +2,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$62]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$62] <- [{"customer_id": $$95, "year": $$96, "month": $$97, "day": $$98, "company": $$99}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        select (and(eq($$95, 1), eq($$96, 2001), eq($$97, 1), eq($$98, 1), eq($$99, "ford"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- STREAM_SELECT  |PARTITIONED|
-          project ([$$99, $$98, $$97, $$96, $$95]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STREAM_PROJECT  |PARTITIONED|
-            assign [$$99, $$98, $$97, $$96, $$95] <- [string-default-null($$m.getField("company")), int64-default-null($$m.getField("day")), int64-default-null($$m.getField("month")), int64-default-null($$m.getField("year")), int64-default-null($$m.getField("customer_id"))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ASSIGN  |PARTITIONED|
+    assign [$$62] <- [{"customer_id": $$95, "year": $$96, "month": $$97, "day": $$98, "company": $$99}] project: [$$62] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (and(eq($$95, 1), eq($$96, 2001), eq($$97, 1), eq($$98, 1), eq($$99, "ford"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        assign [$$99, $$98, $$97, $$96, $$95] <- [string-default-null($$m.getField("company")), int64-default-null($$m.getField("day")), int64-default-null($$m.getField("month")), int64-default-null($$m.getField("year")), int64-default-null($$m.getField("customer_id"))] project: [$$99, $$98, $$97, $$96, $$95] [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]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            data-scan []<-[$$m] <- test.maintenance prefix-filter on: and(eq(int64-default-null($$m.getField("customer_id")), 1), eq(int64-default-null($$m.getField("year")), 2001), eq(int64-default-null($$m.getField("month")), 1), eq(int64-default-null($$m.getField("day")), 1), eq(string-default-null($$m.getField("company")), "ford")) embed-filter-value: true [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|
-                data-scan []<-[$$m] <- test.maintenance prefix-filter on: and(eq(int64-default-null($$m.getField("customer_id")), 1), eq(int64-default-null($$m.getField("year")), 2001), eq(int64-default-null($$m.getField("month")), 1), eq(int64-default-null($$m.getField("day")), 1), eq(string-default-null($$m.getField("company")), "ford")) embed-filter-value: true [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|
\ No newline at end of file
+                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/external-dataset/common/dynamic-prefixes/avro/views/result.004.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/views/result.004.plan
index f7cba8a..70fcea8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/views/result.004.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/avro/views/result.004.plan
@@ -2,21 +2,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$62]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      select (and(eq($$62.getField("customerId"), 1), eq($$94, 2001), eq($$95, 1), eq($$96, 1), eq($$97, "ford"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- STREAM_SELECT  |PARTITIONED|
-        project ([$$97, $$96, $$95, $$94, $$62]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          assign [$$62] <- [{"customer_id": int64-default-null($$m.getField("customer_id")), "year": $$94, "month": $$95, "day": $$96, "company": $$97}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- ASSIGN  |PARTITIONED|
-            assign [$$97, $$96, $$95, $$94] <- [string-default-null($$m.getField("company")), int64-default-null($$m.getField("day")), int64-default-null($$m.getField("month")), int64-default-null($$m.getField("year"))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ASSIGN  |PARTITIONED|
+    select (and(eq($$62.getField("customerId"), 1), eq($$94, 2001), eq($$95, 1), eq($$96, 1), eq($$97, "ford"))) project: [$$62] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- STREAM_SELECT  |PARTITIONED|
+      assign [$$62] <- [{"customer_id": int64-default-null($$m.getField("customer_id")), "year": $$94, "month": $$95, "day": $$96, "company": $$97}] project: [$$97, $$96, $$95, $$94, $$62] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- ASSIGN  |PARTITIONED|
+        assign [$$97, $$96, $$95, $$94] <- [string-default-null($$m.getField("company")), int64-default-null($$m.getField("day")), int64-default-null($$m.getField("month")), int64-default-null($$m.getField("year"))] [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]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            data-scan []<-[$$m] <- test.maintenance2 prefix-filter on: and(eq(int64-default-null($$m.getField("year")), 2001), eq(int64-default-null($$m.getField("month")), 1), eq(int64-default-null($$m.getField("day")), 1), eq(string-default-null($$m.getField("company")), "ford")) embed-filter-value: true [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|
-                data-scan []<-[$$m] <- test.maintenance2 prefix-filter on: and(eq(int64-default-null($$m.getField("year")), 2001), eq(int64-default-null($$m.getField("month")), 1), eq(int64-default-null($$m.getField("day")), 1), eq(string-default-null($$m.getField("company")), "ford")) embed-filter-value: true [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|
+                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/external-dataset/common/dynamic-prefixes/embed-one-value/one-field.131.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/embed-one-value/one-field.131.plan
index 4edf24e..afef4ab 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/embed-one-value/one-field.131.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/embed-one-value/one-field.131.plan
@@ -10,17 +10,15 @@
         -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$28]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select (and(eq(lowercase($$25.getField("first")), "john"), eq(string-concat(ordered-list-constructor("john ", lowercase($$25.getField("last")))), "john smith"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                assign [$$28, $$25] <- [$$d.getField("id"), $$d.getField("name")] [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]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$d] <- test.LastName prefix-filter on: eq(string-concat(ordered-list-constructor("john ", lowercase($$d.getField("name").getField("last")))), "john smith") embed-filter-value: true [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|
+            select (and(eq(lowercase($$25.getField("first")), "john"), eq(string-concat(ordered-list-constructor("john ", lowercase($$25.getField("last")))), "john smith"))) project: [$$d, $$28] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$28, $$25] <- [$$d.getField("id"), $$d.getField("name")] [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]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$d] <- test.LastName prefix-filter on: eq(string-concat(ordered-list-constructor("john ", lowercase($$d.getField("name").getField("last")))), "john smith") embed-filter-value: true [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/external-dataset/common/dynamic-prefixes/not-in/not-in.011.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/not-in/not-in.011.plan
index f5b9fd0..a035869 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/not-in/not-in.011.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/not-in/not-in.011.plan
@@ -10,28 +10,26 @@
         -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$22]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select ($$19) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$d, $$22, $$19]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  subplan {
-                            aggregate [$$19] <- [empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- AGGREGATE  |LOCAL|
-                              select (not(if-missing-or-null(neq(lowercase($$21), "engineering"), false))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_SELECT  |LOCAL|
-                                nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                         } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- SUBPLAN  |PARTITIONED|
-                    assign [$$22, $$21] <- [$$d.getField("id"), $$d.getField("department")] [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]
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        data-scan []<-[$$d] <- test.Department [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|
+            select ($$19) project: [$$d, $$22] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$d, $$22, $$19]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_PROJECT  |PARTITIONED|
+                subplan {
+                          aggregate [$$19] <- [empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- AGGREGATE  |LOCAL|
+                            select (not(if-missing-or-null(neq(lowercase($$21), "engineering"), false))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- STREAM_SELECT  |LOCAL|
+                              nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- SUBPLAN  |PARTITIONED|
+                  assign [$$22, $$21] <- [$$d.getField("id"), $$d.getField("department")] [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]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$d] <- test.Department [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/external-dataset/common/dynamic-prefixes/not-in/not-in.111.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/not-in/not-in.111.plan
index 9bf7d82..f0a3dbe 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/not-in/not-in.111.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/not-in/not-in.111.plan
@@ -10,54 +10,52 @@
         -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$22]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select ($$19) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$19, $$22, $$d]) [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|
-                    group by ([$$25 := $$23]) decor ([$$22; $$d]) {
-                              aggregate [$$19] <- [empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- AGGREGATE  |LOCAL|
-                                select (not(is-missing($$24))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |LOCAL|
-                                  nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                           } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- PRE_CLUSTERED_GROUP_BY[$$23]  |PARTITIONED|
-                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        order (ASC, $$23) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
-                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- HASH_PARTITION_EXCHANGE [$$23]  |PARTITIONED|
-                            project ([$$d, $$22, $$24, $$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|
-                                left outer join (not(if-missing-or-null(neq(lowercase($$21), $#1), false))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- NESTED_LOOP  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    running-aggregate [$$23] <- [create-query-uid()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- RUNNING_AGGREGATE  |PARTITIONED|
-                                      assign [$$22, $$21] <- [$$d.getField("id"), $$d.getField("department")] [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]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          data-scan []<-[$$d] <- test.Department [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]
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                    assign [$$24] <- [true] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- ASSIGN  |UNPARTITIONED|
-                                      unnest $#1 <- scan-collection(array: [ "engineering", "accounting" ]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- UNNEST  |UNPARTITIONED|
-                                        empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+            select ($$19) project: [$$d, $$22] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$19, $$22, $$d]) [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|
+                  group by ([$$25 := $$23]) decor ([$$22; $$d]) {
+                            aggregate [$$19] <- [empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- AGGREGATE  |LOCAL|
+                              select (not(is-missing($$24))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- STREAM_SELECT  |LOCAL|
+                                nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                         } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- PRE_CLUSTERED_GROUP_BY[$$23]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      order (ASC, $$23) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- HASH_PARTITION_EXCHANGE [$$23]  |PARTITIONED|
+                          project ([$$d, $$22, $$24, $$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|
+                              left outer join (not(if-missing-or-null(neq(lowercase($$21), $#1), false))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- NESTED_LOOP  |PARTITIONED|
+                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  running-aggregate [$$23] <- [create-query-uid()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- RUNNING_AGGREGATE  |PARTITIONED|
+                                    assign [$$22, $$21] <- [$$d.getField("id"), $$d.getField("department")] [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]
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        data-scan []<-[$$d] <- test.Department [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]
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  assign [$$24] <- [true] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- ASSIGN  |UNPARTITIONED|
+                                    unnest $#1 <- scan-collection(array: [ "engineering", "accounting" ]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- UNNEST  |UNPARTITIONED|
+                                      empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/one-field/one-field.073.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/one-field/one-field.073.plan
index 02693c3..f8b544d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/one-field/one-field.073.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/one-field/one-field.073.plan
@@ -10,17 +10,15 @@
         -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$27]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select (or(or(eq($$30, "hr"), eq($$30, "accounting")), eq($$d.getField("name").getField("last"), "Smith"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                assign [$$30, $$27] <- [lowercase($$d.getField("department")), $$d.getField("id")] [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]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$d] <- test.Department [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|
+            select (or(or(eq($$30, "hr"), eq($$30, "accounting")), eq($$d.getField("name").getField("last"), "Smith"))) project: [$$d, $$27] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$30, $$27] <- [lowercase($$d.getField("department")), $$d.getField("id")] [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]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$d] <- test.Department [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/external-dataset/common/dynamic-prefixes/one-field/one-field.077.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/one-field/one-field.077.plan
index 3a8b00d..2015476 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/one-field/one-field.077.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/one-field/one-field.077.plan
@@ -10,17 +10,15 @@
         -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$28]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select (or(eq($$31, "hr"), eq($$31, "accounting"), eq($$d.getField("name").getField("last"), "Smith"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                assign [$$31, $$28] <- [lowercase($$d.getField("department")), $$d.getField("id")] [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]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$d] <- test.Department [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|
+            select (or(eq($$31, "hr"), eq($$31, "accounting"), eq($$d.getField("name").getField("last"), "Smith"))) project: [$$d, $$28] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$31, $$28] <- [lowercase($$d.getField("department")), $$d.getField("id")] [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]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$d] <- test.Department [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/external-dataset/common/dynamic-prefixes/one-field/one-field.131.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/one-field/one-field.131.plan
index 44fd334..929d06d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/one-field/one-field.131.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/one-field/one-field.131.plan
@@ -10,17 +10,15 @@
         -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$28]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select (and(eq(lowercase($$25.getField("first")), "john"), eq(string-concat(ordered-list-constructor("john ", lowercase($$25.getField("last")))), "john smith"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                assign [$$28, $$25] <- [$$d.getField("id"), $$d.getField("name")] [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]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$d] <- test.LastName prefix-filter on: eq(string-concat(ordered-list-constructor("john ", lowercase($$d.getField("name").getField("last")))), "john smith") [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|
+            select (and(eq(lowercase($$25.getField("first")), "john"), eq(string-concat(ordered-list-constructor("john ", lowercase($$25.getField("last")))), "john smith"))) project: [$$d, $$28] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$28, $$25] <- [$$d.getField("id"), $$d.getField("name")] [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]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$d] <- test.LastName prefix-filter on: eq(string-concat(ordered-list-constructor("john ", lowercase($$d.getField("name").getField("last")))), "john smith") [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/external-dataset/common/dynamic-prefixes/one-field/one-field.211.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/one-field/one-field.211.plan
index 646bb04..fd9f409 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/one-field/one-field.211.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/one-field/one-field.211.plan
@@ -26,42 +26,38 @@
                         -- HYBRID_HASH_JOIN [$$39][$$43]  |PARTITIONED|
                           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            project ([$$d, $$45, $$34, $$39]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              assign [$$39, $$34] <- [lowercase($$48), lowercase($$d.getField("department"))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ASSIGN  |PARTITIONED|
-                                project ([$$d, $$45, $$48]) [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|
-                                    join (eq($$45, $$46)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- HYBRID_HASH_JOIN [$$45][$$46]  |PARTITIONED|
-                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- HASH_PARTITION_EXCHANGE [$$45]  |PARTITIONED|
-                                        assign [$$45] <- [$$d.getField("id")] [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]
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            data-scan []<-[$$d] <- test.Department prefix-filter on: or(eq(lowercase($$d.getField("department")), "hr"), eq(lowercase($$d.getField("department")), "accounting")) [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 [$$46]  |PARTITIONED|
-                                        project ([$$48, $$46]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          assign [$$48, $$46] <- [$$l.getField("name").getField("last"), $$l.getField("id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- ASSIGN  |PARTITIONED|
+                            assign [$$39, $$34] <- [lowercase($$48), lowercase($$d.getField("department"))] project: [$$d, $$45, $$34, $$39] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- ASSIGN  |PARTITIONED|
+                              project ([$$d, $$45, $$48]) [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|
+                                  join (eq($$45, $$46)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- HYBRID_HASH_JOIN [$$45][$$46]  |PARTITIONED|
+                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- HASH_PARTITION_EXCHANGE [$$45]  |PARTITIONED|
+                                      assign [$$45] <- [$$d.getField("id")] [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]
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          data-scan []<-[$$d] <- test.Department prefix-filter on: or(eq(lowercase($$d.getField("department")), "hr"), eq(lowercase($$d.getField("department")), "accounting")) [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|
-                                              data-scan []<-[$$l] <- test.LastName prefix-filter on: or(eq(lowercase($$l.getField("name").getField("last")), "miller"), eq(lowercase($$l.getField("name").getField("last")), "smith")) [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|
+                                              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 [$$46]  |PARTITIONED|
+                                      assign [$$48, $$46] <- [$$l.getField("name").getField("last"), $$l.getField("id")] project: [$$48, $$46] [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]
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          data-scan []<-[$$l] <- test.LastName prefix-filter on: or(eq(lowercase($$l.getField("name").getField("last")), "miller"), eq(lowercase($$l.getField("name").getField("last")), "smith")) [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]
                           -- BROADCAST_EXCHANGE  |PARTITIONED|
                             unnest $$43 <- scan-collection(array: [ "miller", "smith" ]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/one-field/one-field.221.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/one-field/one-field.221.plan
index 6b83b9e..1c924a2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/one-field/one-field.221.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/one-field/one-field.221.plan
@@ -38,20 +38,18 @@
                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                           -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
-                            project ([$$42]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              select (eq(lowercase($$l.getField("name").getField("last")), "smith")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_SELECT  |PARTITIONED|
-                                assign [$$42] <- [$$l.getField("id")] [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]
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    data-scan []<-[$$l] <- test.LastName prefix-filter on: eq(lowercase($$l.getField("name").getField("last")), "smith") [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|
+                            select (eq(lowercase($$l.getField("name").getField("last")), "smith")) project: [$$42] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- STREAM_SELECT  |PARTITIONED|
+                              assign [$$42] <- [$$l.getField("id")] [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]
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  data-scan []<-[$$l] <- test.LastName prefix-filter on: eq(lowercase($$l.getField("name").getField("last")), "smith") [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]
                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                     unnest $$40 <- scan-collection(array: [ "hr", "accounting" ]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/one-field/one-field.231.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/one-field/one-field.231.plan
index c6a4c95..ad9c368 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/one-field/one-field.231.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/one-field/one-field.231.plan
@@ -10,39 +10,35 @@
         -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$42]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select (or(or(eq($$32, "hr"), eq($$32, "accounting")), eq($$41, "smith"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$d, $$32, $$42, $$41]) [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|
-                    join (eq($$42, $$43)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HYBRID_HASH_JOIN [$$42][$$43]  |PARTITIONED|
-                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
-                        assign [$$32, $$42] <- [lowercase($$d.getField("department")), $$d.getField("id")] [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]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            data-scan []<-[$$d] <- test.Department [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 [$$43]  |PARTITIONED|
-                        project ([$$41, $$43]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$41, $$43] <- [lowercase($$l.getField("name").getField("last")), $$l.getField("id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- ASSIGN  |PARTITIONED|
+            select (or(or(eq($$32, "hr"), eq($$32, "accounting")), eq($$41, "smith"))) project: [$$d, $$42] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$d, $$32, $$42, $$41]) [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|
+                  join (eq($$42, $$43)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HYBRID_HASH_JOIN [$$42][$$43]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
+                      assign [$$32, $$42] <- [lowercase($$d.getField("department")), $$d.getField("id")] [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]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$d] <- test.Department [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|
-                              data-scan []<-[$$l] <- test.LastName [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|
+                              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 [$$43]  |PARTITIONED|
+                      assign [$$41, $$43] <- [lowercase($$l.getField("name").getField("last")), $$l.getField("id")] project: [$$41, $$43] [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]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$l] <- test.LastName [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/external-dataset/common/dynamic-prefixes/parquet/embed-one-value/one-field.131.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/parquet/embed-one-value/one-field.131.plan
index 4edf24e..afef4ab 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/parquet/embed-one-value/one-field.131.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/parquet/embed-one-value/one-field.131.plan
@@ -10,17 +10,15 @@
         -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$28]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select (and(eq(lowercase($$25.getField("first")), "john"), eq(string-concat(ordered-list-constructor("john ", lowercase($$25.getField("last")))), "john smith"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                assign [$$28, $$25] <- [$$d.getField("id"), $$d.getField("name")] [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]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$d] <- test.LastName prefix-filter on: eq(string-concat(ordered-list-constructor("john ", lowercase($$d.getField("name").getField("last")))), "john smith") embed-filter-value: true [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|
+            select (and(eq(lowercase($$25.getField("first")), "john"), eq(string-concat(ordered-list-constructor("john ", lowercase($$25.getField("last")))), "john smith"))) project: [$$d, $$28] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$28, $$25] <- [$$d.getField("id"), $$d.getField("name")] [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]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$d] <- test.LastName prefix-filter on: eq(string-concat(ordered-list-constructor("john ", lowercase($$d.getField("name").getField("last")))), "john smith") embed-filter-value: true [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/external-dataset/common/dynamic-prefixes/parquet/one-field/one-field.131.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/parquet/one-field/one-field.131.plan
index 44fd334..929d06d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/parquet/one-field/one-field.131.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/parquet/one-field/one-field.131.plan
@@ -10,17 +10,15 @@
         -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$28]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select (and(eq(lowercase($$25.getField("first")), "john"), eq(string-concat(ordered-list-constructor("john ", lowercase($$25.getField("last")))), "john smith"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                assign [$$28, $$25] <- [$$d.getField("id"), $$d.getField("name")] [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]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$d] <- test.LastName prefix-filter on: eq(string-concat(ordered-list-constructor("john ", lowercase($$d.getField("name").getField("last")))), "john smith") [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|
+            select (and(eq(lowercase($$25.getField("first")), "john"), eq(string-concat(ordered-list-constructor("john ", lowercase($$25.getField("last")))), "john smith"))) project: [$$d, $$28] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$28, $$25] <- [$$d.getField("id"), $$d.getField("name")] [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]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$d] <- test.LastName prefix-filter on: eq(string-concat(ordered-list-constructor("john ", lowercase($$d.getField("name").getField("last")))), "john smith") [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/external-dataset/common/dynamic-prefixes/parquet/views/result.002.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/parquet/views/result.002.adm
index ece3177..4b4dc7c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/parquet/views/result.002.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/parquet/views/result.002.adm
@@ -2,21 +2,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$62]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$62] <- [{"customer_id": $$95, "year": $$96, "month": $$97, "day": $$98, "company": $$99}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        select (and(eq($$95, 1), eq($$96, 2001), eq($$97, 1), eq($$98, 1), eq($$99, "ford"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- STREAM_SELECT  |PARTITIONED|
-          project ([$$99, $$98, $$97, $$96, $$95]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STREAM_PROJECT  |PARTITIONED|
-            assign [$$99, $$98, $$97, $$96, $$95] <- [string-default-null($$m.getField("company")), int64-default-null($$m.getField("day")), int64-default-null($$m.getField("month")), int64-default-null($$m.getField("year")), int64-default-null($$m.getField("customer_id"))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ASSIGN  |PARTITIONED|
+    assign [$$62] <- [{"customer_id": $$95, "year": $$96, "month": $$97, "day": $$98, "company": $$99}] project: [$$62] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (and(eq($$95, 1), eq($$96, 2001), eq($$97, 1), eq($$98, 1), eq($$99, "ford"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        assign [$$99, $$98, $$97, $$96, $$95] <- [string-default-null($$m.getField("company")), int64-default-null($$m.getField("day")), int64-default-null($$m.getField("month")), int64-default-null($$m.getField("year")), int64-default-null($$m.getField("customer_id"))] project: [$$99, $$98, $$97, $$96, $$95] [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]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            data-scan []<-[$$m] <- test.maintenance project ({month:any,year:any,company:any,customer_id:any,day:any}) prefix-filter on: and(eq(int64-default-null($$m.getField("customer_id")), 1), eq(int64-default-null($$m.getField("year")), 2001), eq(int64-default-null($$m.getField("month")), 1), eq(int64-default-null($$m.getField("day")), 1), eq(string-default-null($$m.getField("company")), "ford")) embed-filter-value: true [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|
-                data-scan []<-[$$m] <- test.maintenance project ({month:any,year:any,company:any,customer_id:any,day:any}) prefix-filter on: and(eq(int64-default-null($$m.getField("customer_id")), 1), eq(int64-default-null($$m.getField("year")), 2001), eq(int64-default-null($$m.getField("month")), 1), eq(int64-default-null($$m.getField("day")), 1), eq(string-default-null($$m.getField("company")), "ford")) embed-filter-value: true [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|
\ No newline at end of file
+                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/external-dataset/common/dynamic-prefixes/parquet/views/result.004.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/parquet/views/result.004.adm
index 4c78355..2545d66 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/parquet/views/result.004.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/parquet/views/result.004.adm
@@ -2,21 +2,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$62]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      select (and(eq($$62.getField("customerId"), 1), eq($$94, 2001), eq($$95, 1), eq($$96, 1), eq($$97, "ford"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- STREAM_SELECT  |PARTITIONED|
-        project ([$$97, $$96, $$95, $$94, $$62]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          assign [$$62] <- [{"customer_id": int64-default-null($$m.getField("customer_id")), "year": $$94, "month": $$95, "day": $$96, "company": $$97}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- ASSIGN  |PARTITIONED|
-            assign [$$97, $$96, $$95, $$94] <- [string-default-null($$m.getField("company")), int64-default-null($$m.getField("day")), int64-default-null($$m.getField("month")), int64-default-null($$m.getField("year"))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ASSIGN  |PARTITIONED|
+    select (and(eq($$62.getField("customerId"), 1), eq($$94, 2001), eq($$95, 1), eq($$96, 1), eq($$97, "ford"))) project: [$$62] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- STREAM_SELECT  |PARTITIONED|
+      assign [$$62] <- [{"customer_id": int64-default-null($$m.getField("customer_id")), "year": $$94, "month": $$95, "day": $$96, "company": $$97}] project: [$$97, $$96, $$95, $$94, $$62] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- ASSIGN  |PARTITIONED|
+        assign [$$97, $$96, $$95, $$94] <- [string-default-null($$m.getField("company")), int64-default-null($$m.getField("day")), int64-default-null($$m.getField("month")), int64-default-null($$m.getField("year"))] [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]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            data-scan []<-[$$m] <- test.maintenance2 project ({month:any,year:any,company:any,customer_id:any,day:any}) prefix-filter on: and(eq(int64-default-null($$m.getField("year")), 2001), eq(int64-default-null($$m.getField("month")), 1), eq(int64-default-null($$m.getField("day")), 1), eq(string-default-null($$m.getField("company")), "ford")) embed-filter-value: true [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|
-                data-scan []<-[$$m] <- test.maintenance2 project ({month:any,year:any,company:any,customer_id:any,day:any}) prefix-filter on: and(eq(int64-default-null($$m.getField("year")), 2001), eq(int64-default-null($$m.getField("month")), 1), eq(int64-default-null($$m.getField("day")), 1), eq(string-default-null($$m.getField("company")), "ford")) embed-filter-value: true [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|
+                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/external-dataset/common/dynamic-prefixes/views/result.002.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/views/result.002.plan
index 2478db2..eaa9ad6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/views/result.002.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/views/result.002.plan
@@ -2,21 +2,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$62]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$62] <- [{"customer_id": $$95, "year": $$96, "month": $$97, "day": $$98, "company": $$99}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        select (and(eq($$95, 1), eq($$96, 2001), eq($$97, 1), eq($$98, 1), eq($$99, "ford"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- STREAM_SELECT  |PARTITIONED|
-          project ([$$99, $$98, $$97, $$96, $$95]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STREAM_PROJECT  |PARTITIONED|
-            assign [$$99, $$98, $$97, $$96, $$95] <- [string-default-null($$m.getField("company")), int64-default-null($$m.getField("day")), int64-default-null($$m.getField("month")), int64-default-null($$m.getField("year")), int64-default-null($$m.getField("customer_id"))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ASSIGN  |PARTITIONED|
+    assign [$$62] <- [{"customer_id": $$95, "year": $$96, "month": $$97, "day": $$98, "company": $$99}] project: [$$62] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (and(eq($$95, 1), eq($$96, 2001), eq($$97, 1), eq($$98, 1), eq($$99, "ford"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        assign [$$99, $$98, $$97, $$96, $$95] <- [string-default-null($$m.getField("company")), int64-default-null($$m.getField("day")), int64-default-null($$m.getField("month")), int64-default-null($$m.getField("year")), int64-default-null($$m.getField("customer_id"))] project: [$$99, $$98, $$97, $$96, $$95] [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]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            data-scan []<-[$$m] <- test.maintenance prefix-filter on: and(eq(int64-default-null($$m.getField("customer_id")), 1), eq(int64-default-null($$m.getField("year")), 2001), eq(int64-default-null($$m.getField("month")), 1), eq(int64-default-null($$m.getField("day")), 1), eq(string-default-null($$m.getField("company")), "ford")) embed-filter-value: true [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|
-                data-scan []<-[$$m] <- test.maintenance prefix-filter on: and(eq(int64-default-null($$m.getField("customer_id")), 1), eq(int64-default-null($$m.getField("year")), 2001), eq(int64-default-null($$m.getField("month")), 1), eq(int64-default-null($$m.getField("day")), 1), eq(string-default-null($$m.getField("company")), "ford")) embed-filter-value: true [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|
\ No newline at end of file
+                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/external-dataset/common/dynamic-prefixes/views/result.004.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/views/result.004.plan
index f7cba8a..70fcea8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/views/result.004.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/dynamic-prefixes/views/result.004.plan
@@ -2,21 +2,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$62]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      select (and(eq($$62.getField("customerId"), 1), eq($$94, 2001), eq($$95, 1), eq($$96, 1), eq($$97, "ford"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- STREAM_SELECT  |PARTITIONED|
-        project ([$$97, $$96, $$95, $$94, $$62]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          assign [$$62] <- [{"customer_id": int64-default-null($$m.getField("customer_id")), "year": $$94, "month": $$95, "day": $$96, "company": $$97}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- ASSIGN  |PARTITIONED|
-            assign [$$97, $$96, $$95, $$94] <- [string-default-null($$m.getField("company")), int64-default-null($$m.getField("day")), int64-default-null($$m.getField("month")), int64-default-null($$m.getField("year"))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ASSIGN  |PARTITIONED|
+    select (and(eq($$62.getField("customerId"), 1), eq($$94, 2001), eq($$95, 1), eq($$96, 1), eq($$97, "ford"))) project: [$$62] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- STREAM_SELECT  |PARTITIONED|
+      assign [$$62] <- [{"customer_id": int64-default-null($$m.getField("customer_id")), "year": $$94, "month": $$95, "day": $$96, "company": $$97}] project: [$$97, $$96, $$95, $$94, $$62] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- ASSIGN  |PARTITIONED|
+        assign [$$97, $$96, $$95, $$94] <- [string-default-null($$m.getField("company")), int64-default-null($$m.getField("day")), int64-default-null($$m.getField("month")), int64-default-null($$m.getField("year"))] [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]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            data-scan []<-[$$m] <- test.maintenance2 prefix-filter on: and(eq(int64-default-null($$m.getField("year")), 2001), eq(int64-default-null($$m.getField("month")), 1), eq(int64-default-null($$m.getField("day")), 1), eq(string-default-null($$m.getField("company")), "ford")) embed-filter-value: true [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|
-                data-scan []<-[$$m] <- test.maintenance2 prefix-filter on: and(eq(int64-default-null($$m.getField("year")), 2001), eq(int64-default-null($$m.getField("month")), 1), eq(int64-default-null($$m.getField("day")), 1), eq(string-default-null($$m.getField("company")), "ford")) embed-filter-value: true [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|
+                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/external-dataset/common/json/json/external_dataset.011.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.011.plan
index e69fd7c..3b3f2c1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.011.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.011.plan
@@ -1,48 +1,42 @@
-distribute result [$$48] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$50] [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|
-    project ([$$48]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$48] <- [{"cnt": $$51}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$51] <- [agg-sql-sum($$53)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$53] <- [agg-sql-count($$49)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              project ([$$49]) [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|
-                  join (eq($$49, $$50)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HYBRID_HASH_JOIN [$$49][$$50]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HASH_PARTITION_EXCHANGE [$$49]  |PARTITIONED|
-                      project ([$$49]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$49] <- [$$a.getField("quarter")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ASSIGN  |PARTITIONED|
+    assign [$$50] <- [{"cnt": $$53}] project: [$$50] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$53] <- [agg-sql-sum($$55)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$55] <- [agg-sql-count($$51)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            project ([$$51]) [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|
+                join (eq($$51, $$52)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HYBRID_HASH_JOIN [$$51][$$52]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HASH_PARTITION_EXCHANGE [$$51]  |PARTITIONED|
+                    assign [$$51] <- [$$a.getField("quarter")] project: [$$51] [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]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        data-scan []<-[$$a] <- test.test6 [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|
-                            data-scan []<-[$$a] <- test.test6 [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 [$$50]  |PARTITIONED|
-                      project ([$$50]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$50] <- [$$b.getField("quarter")] [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|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
+                    assign [$$52] <- [$$b.getField("quarter")] project: [$$52] [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]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        data-scan []<-[$$b] <- test.test6 [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|
-                            data-scan []<-[$$b] <- test.test6 [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|
+                            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/external-dataset/common/json/json/external_dataset.013.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.013.plan
index 925b4f1..0192431 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.013.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/json/json/external_dataset.013.plan
@@ -1,56 +1,32 @@
-distribute result [$$47] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$50] [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|
-    project ([$$47]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$47] <- [{"cnt": $$50}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$50] <- [agg-sql-sum($$52)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$52] <- [agg-sql-count($$48)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              project ([$$48]) [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|
-                  join (eq($$48, $$49)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HYBRID_HASH_JOIN [$$48][$$49]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      project ([$$48]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$48] <- [$$49] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ASSIGN  |PARTITIONED|
+    assign [$$50] <- [{"cnt": $$53}] project: [$$50] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$53] <- [agg-sql-sum($$55)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$55] <- [agg-sql-count($$51)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            project ([$$51]) [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|
+                join (eq($$51, $$52)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HYBRID_HASH_JOIN [$$51][$$52]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    assign [$$51] <- [$$52] project: [$$51] [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]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- REPLICATE  |PARTITIONED|
                           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- REPLICATE  |PARTITIONED|
-                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- HASH_PARTITION_EXCHANGE [$$49]  |PARTITIONED|
-                                project ([$$49]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  assign [$$49] <- [$$b.getField("quarter")] [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]
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      data-scan []<-[$$b] <- test.test6 [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  |PARTITIONED|
-                      replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- REPLICATE  |PARTITIONED|
-                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HASH_PARTITION_EXCHANGE [$$49]  |PARTITIONED|
-                          project ([$$49]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            assign [$$49] <- [$$b.getField("quarter")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
+                            assign [$$52] <- [$$b.getField("quarter")] project: [$$52] [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]
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -60,3 +36,19 @@
                                   -- 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  |PARTITIONED|
+                    replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- REPLICATE  |PARTITIONED|
+                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
+                        assign [$$52] <- [$$b.getField("quarter")] project: [$$52] [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]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            data-scan []<-[$$b] <- test.test6 [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/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.03.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.03.plan
index 7390e2a..027926b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.03.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.03.plan
@@ -1,28 +1,24 @@
-distribute result [$$17] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$18] [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 ([$$17]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$17] <- [{"display_url": $$20}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$20]) [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 [$$19(ASC) ]  |PARTITIONED|
-            order (ASC, $$19) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$20, $$19]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$20, $$19] <- [get-item($$p.getField("entities").getField("urls"), 0).getField("display_url"), $$p.getField("id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ASSIGN  |PARTITIONED|
+    assign [$$18] <- [{"display_url": $$21}] project: [$$18] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$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]
+        -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
+          order (ASC, $$20) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$21, $$20] <- [get-item($$p.getField("entities").getField("urls"), 0).getField("display_url"), $$p.getField("id")] project: [$$21, $$20] [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]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$p] <- test.ParquetDataset [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|
-                      data-scan []<-[$$p] <- test.ParquetDataset [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|
+                      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/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.05.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.05.plan
index d18185b..2161117 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.05.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.05.plan
@@ -1,28 +1,24 @@
-distribute result [$$17] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$18] [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 ([$$17]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$17] <- [{"display_url": $$20}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$20]) [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 [$$19(ASC) ]  |PARTITIONED|
-            order (ASC, $$19) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$20, $$19]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$20, $$19] <- [get-item($$p.getField("entities").getField("urls"), 0).getField("display_url"), $$p.getField("id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ASSIGN  |PARTITIONED|
+    assign [$$18] <- [{"display_url": $$21}] project: [$$18] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$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]
+        -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
+          order (ASC, $$20) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$21, $$20] <- [get-item($$p.getField("entities").getField("urls"), 0).getField("display_url"), $$p.getField("id")] project: [$$21, $$20] [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]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$p] <- test.ParquetDataset project ({entities:{urls:[{display_url:any}]},id:any}) [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|
-                      data-scan []<-[$$p] <- test.ParquetDataset project ({entities:{urls:[{display_url:any}]},id:any}) [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|
+                      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/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.07.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.07.plan
index d9100d1..2812ab8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.07.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.07.plan
@@ -1,34 +1,28 @@
-distribute result [$$21] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$22] [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 ([$$21]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$21] <- [{"display_url": $$25}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$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]
-          -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-            order (ASC, $$24) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$25, $$24]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$25] <- [array-star($$23).getField("display_url")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$22] <- [{"display_url": $$26}] project: [$$22] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$26]) [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 [$$25(ASC) ]  |PARTITIONED|
+          order (ASC, $$25) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$26] <- [array-star($$24).getField("display_url")] project: [$$26, $$25] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ASSIGN  |PARTITIONED|
+                select (not(is-missing($$24))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STREAM_SELECT  |PARTITIONED|
+                  assign [$$24, $$25] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")] project: [$$24, $$25] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- ASSIGN  |PARTITIONED|
-                    select (not(is-missing($$23))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_SELECT  |PARTITIONED|
-                      project ([$$23, $$24]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$23, $$24] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")] [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]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            data-scan []<-[$$p] <- test.ParquetDataset [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  |PARTITIONED|
+                      data-scan []<-[$$p] <- test.ParquetDataset [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/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.09.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.09.plan
index 3d402ee..940db43 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.09.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.09.plan
@@ -1,34 +1,28 @@
-distribute result [$$21] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$22] [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 ([$$21]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$21] <- [{"display_url": $$25}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$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]
-          -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-            order (ASC, $$24) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$25, $$24]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$25] <- [array-star($$23).getField("display_url")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$22] <- [{"display_url": $$26}] project: [$$22] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$26]) [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 [$$25(ASC) ]  |PARTITIONED|
+          order (ASC, $$25) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$26] <- [array-star($$24).getField("display_url")] project: [$$26, $$25] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ASSIGN  |PARTITIONED|
+                select (not(is-missing($$24))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STREAM_SELECT  |PARTITIONED|
+                  assign [$$24, $$25] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")] project: [$$24, $$25] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- ASSIGN  |PARTITIONED|
-                    select (not(is-missing($$23))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_SELECT  |PARTITIONED|
-                      project ([$$23, $$24]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$23, $$24] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")] [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]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            data-scan []<-[$$p] <- test.ParquetDataset project ({entities:{urls:[{display_url:any}]},id:any}) [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  |PARTITIONED|
+                      data-scan []<-[$$p] <- test.ParquetDataset project ({entities:{urls:[{display_url:any}]},id:any}) [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/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.11.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.11.plan
index 4b19e00..58a0b66 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.11.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.11.plan
@@ -1,32 +1,26 @@
-distribute result [$$26] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$28] [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 ([$$26]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$26] <- [{"display_url": $$urls.getField("display_url")}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$urls]) [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 [$$29(ASC) ]  |PARTITIONED|
-            order (ASC, $$29) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$urls, $$29]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  unnest $$urls <- scan-collection($$28) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- UNNEST  |PARTITIONED|
-                    project ([$$28, $$29]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$28, $$29] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")] [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]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$p] <- test.ParquetDataset [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|
+    assign [$$28] <- [{"display_url": $$urls.getField("display_url")}] project: [$$28] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$urls]) [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 [$$31(ASC) ]  |PARTITIONED|
+          order (ASC, $$31) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              unnest $$urls <- scan-collection($$30) project: [$$urls, $$31] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- UNNEST  |PARTITIONED|
+                assign [$$30, $$31] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")] project: [$$30, $$31] [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]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$p] <- test.ParquetDataset [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/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.13.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.13.plan
index fd5eea3..ddefb06 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.13.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.13.plan
@@ -1,32 +1,26 @@
-distribute result [$$26] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$28] [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 ([$$26]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$26] <- [{"display_url": $$urls.getField("display_url")}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$urls]) [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 [$$29(ASC) ]  |PARTITIONED|
-            order (ASC, $$29) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$urls, $$29]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  unnest $$urls <- scan-collection($$28) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- UNNEST  |PARTITIONED|
-                    project ([$$28, $$29]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$28, $$29] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")] [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]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$p] <- test.ParquetDataset project ({entities:{urls:[{display_url:any}]},id:any}) [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|
+    assign [$$28] <- [{"display_url": $$urls.getField("display_url")}] project: [$$28] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$urls]) [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 [$$31(ASC) ]  |PARTITIONED|
+          order (ASC, $$31) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              unnest $$urls <- scan-collection($$30) project: [$$urls, $$31] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- UNNEST  |PARTITIONED|
+                assign [$$30, $$31] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")] project: [$$30, $$31] [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]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$p] <- test.ParquetDataset project ({entities:{urls:[{display_url:any}]},id:any}) [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/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.15.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.15.plan
index 534953d..6d1bbd7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.15.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.15.plan
@@ -1,39 +1,37 @@
-distribute result [$$46] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$49] [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|
-    aggregate [$$46] <- [agg-sql-sum($$52)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    aggregate [$$49] <- [agg-sql-sum($$55)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
     -- AGGREGATE  |UNPARTITIONED|
       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        aggregate [$$52] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        aggregate [$$55] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
         -- AGGREGATE  |PARTITIONED|
-          select ($$39) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          select ($$42) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- STREAM_SELECT  |PARTITIONED|
-            project ([$$39]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            project ([$$42]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
             -- STREAM_PROJECT  |PARTITIONED|
               subplan {
-                        aggregate [$$39] <- [empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        aggregate [$$42] <- [empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- AGGREGATE  |LOCAL|
-                          select (not(if-missing-or-null(eq($$48, "string"), false))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          select (not(if-missing-or-null(eq($$51, "string"), false))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                           -- STREAM_SELECT  |LOCAL|
-                            assign [$$48] <- [$$ht.getField("display_url")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            assign [$$51] <- [$$ht.getField("display_url")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- ASSIGN  |LOCAL|
-                              unnest $$ht <- scan-collection($$47) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              unnest $$ht <- scan-collection($$50) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
               -- SUBPLAN  |PARTITIONED|
-                project ([$$47]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$47] <- [$$p.getField("entities").getField("urls")] [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]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      data-scan []<-[$$p] <- test.ParquetDataset [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|
+                assign [$$50] <- [$$p.getField("entities").getField("urls")] project: [$$50] [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]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$p] <- test.ParquetDataset [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/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.17.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.17.plan
index 70a8f3b..1457164 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.17.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.17.plan
@@ -1,39 +1,37 @@
-distribute result [$$46] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$49] [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|
-    aggregate [$$46] <- [agg-sql-sum($$52)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    aggregate [$$49] <- [agg-sql-sum($$55)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
     -- AGGREGATE  |UNPARTITIONED|
       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        aggregate [$$52] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        aggregate [$$55] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
         -- AGGREGATE  |PARTITIONED|
-          select ($$39) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          select ($$42) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- STREAM_SELECT  |PARTITIONED|
-            project ([$$39]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            project ([$$42]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
             -- STREAM_PROJECT  |PARTITIONED|
               subplan {
-                        aggregate [$$39] <- [empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        aggregate [$$42] <- [empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- AGGREGATE  |LOCAL|
-                          select (not(if-missing-or-null(eq($$48, "string"), false))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          select (not(if-missing-or-null(eq($$51, "string"), false))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                           -- STREAM_SELECT  |LOCAL|
-                            assign [$$48] <- [$$ht.getField("display_url")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            assign [$$51] <- [$$ht.getField("display_url")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- ASSIGN  |LOCAL|
-                              unnest $$ht <- scan-collection($$47) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              unnest $$ht <- scan-collection($$50) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
               -- SUBPLAN  |PARTITIONED|
-                project ([$$47]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$47] <- [$$p.getField("entities").getField("urls")] [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]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      data-scan []<-[$$p] <- test.ParquetDataset project ({entities:{urls:[{display_url:any}]}}) [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|
+                assign [$$50] <- [$$p.getField("entities").getField("urls")] project: [$$50] [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]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$p] <- test.ParquetDataset project ({entities:{urls:[{display_url:any}]}}) [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/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.19.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.19.plan
index 62eeb47..afd8c11 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.19.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.19.plan
@@ -1,28 +1,24 @@
-distribute result [$$18] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$19] [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 ([$$18]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$18] <- [get-item(get-item(get-item($$21, 0), 0), 0)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$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]
-          -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-            order (ASC, $$20) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$21, $$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$21, $$20] <- [$$p.getField("place").getField("bounding_box").getField("coordinates"), $$p.getField("id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ASSIGN  |PARTITIONED|
+    assign [$$19] <- [get-item(get-item(get-item($$22, 0), 0), 0)] project: [$$19] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$22]) [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) ]  |PARTITIONED|
+          order (ASC, $$21) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$22, $$21] <- [$$p.getField("place").getField("bounding_box").getField("coordinates"), $$p.getField("id")] project: [$$22, $$21] [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]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$p] <- test.ParquetDataset [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|
-                      data-scan []<-[$$p] <- test.ParquetDataset [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|
+                      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/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.21.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.21.plan
index 7143a1c..3d6e07a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.21.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.21.plan
@@ -1,28 +1,24 @@
-distribute result [$$18] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$19] [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 ([$$18]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$18] <- [get-item(get-item(get-item($$21, 0), 0), 0)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$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]
-          -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-            order (ASC, $$20) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$21, $$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$21, $$20] <- [$$p.getField("place").getField("bounding_box").getField("coordinates"), $$p.getField("id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ASSIGN  |PARTITIONED|
+    assign [$$19] <- [get-item(get-item(get-item($$22, 0), 0), 0)] project: [$$19] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$22]) [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) ]  |PARTITIONED|
+          order (ASC, $$21) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$22, $$21] <- [$$p.getField("place").getField("bounding_box").getField("coordinates"), $$p.getField("id")] project: [$$22, $$21] [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]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$p] <- test.ParquetDataset project ({place:{bounding_box:{coordinates:[[[any]]]}},id:any}) [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|
-                      data-scan []<-[$$p] <- test.ParquetDataset project ({place:{bounding_box:{coordinates:[[[any]]]}},id:any}) [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|
+                      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/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.07.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.07.plan
index 9d3d10f..69eb459 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.07.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.07.plan
@@ -1,46 +1,42 @@
-distribute result [$$28] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$30] [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 ([$$28]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$28] <- [{"p1": $$p1, "id": $$30}] [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 [$$30(ASC) ]  |PARTITIONED|
-          order (ASC, $$30) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$p1, $$30]) [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|
-                  join (eq($$31, $$30)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HYBRID_HASH_JOIN [$$31][$$30]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
-                      assign [$$31] <- [$$p1.getField("id")] [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]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$p1] <- test.ParquetDataset [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 [$$30]  |PARTITIONED|
-                      project ([$$30]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$30] <- [$$p2.getField("id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ASSIGN  |PARTITIONED|
+    assign [$$30] <- [{"p1": $$p1, "id": $$32}] project: [$$30] [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 [$$32(ASC) ]  |PARTITIONED|
+        order (ASC, $$32) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            project ([$$p1, $$32]) [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|
+                join (eq($$33, $$32)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HYBRID_HASH_JOIN [$$33][$$32]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HASH_PARTITION_EXCHANGE [$$33]  |PARTITIONED|
+                    assign [$$33] <- [$$p1.getField("id")] [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]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        data-scan []<-[$$p1] <- test.ParquetDataset [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|
-                            data-scan []<-[$$p2] <- test.ParquetDataset2 [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|
+                            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 [$$32]  |PARTITIONED|
+                    assign [$$32] <- [$$p2.getField("id")] project: [$$32] [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]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        data-scan []<-[$$p2] <- test.ParquetDataset2 [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/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.09.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.09.plan
index 90f1d05..fddff2f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.09.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.09.plan
@@ -1,46 +1,42 @@
-distribute result [$$28] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$30] [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 ([$$28]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$28] <- [{"p1": $$p1, "id": $$30}] [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 [$$30(ASC) ]  |PARTITIONED|
-          order (ASC, $$30) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$p1, $$30]) [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|
-                  join (eq($$31, $$30)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HYBRID_HASH_JOIN [$$31][$$30]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
-                      assign [$$31] <- [$$p1.getField("id")] [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]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$p1] <- test.ParquetDataset [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 [$$30]  |PARTITIONED|
-                      project ([$$30]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$30] <- [$$p2.getField("id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ASSIGN  |PARTITIONED|
+    assign [$$30] <- [{"p1": $$p1, "id": $$32}] project: [$$30] [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 [$$32(ASC) ]  |PARTITIONED|
+        order (ASC, $$32) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            project ([$$p1, $$32]) [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|
+                join (eq($$33, $$32)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HYBRID_HASH_JOIN [$$33][$$32]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HASH_PARTITION_EXCHANGE [$$33]  |PARTITIONED|
+                    assign [$$33] <- [$$p1.getField("id")] [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]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        data-scan []<-[$$p1] <- test.ParquetDataset [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|
-                            data-scan []<-[$$p2] <- test.ParquetDataset2 project ({id:any}) [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|
+                            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 [$$32]  |PARTITIONED|
+                    assign [$$32] <- [$$p2.getField("id")] project: [$$32] [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]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        data-scan []<-[$$p2] <- test.ParquetDataset2 project ({id:any}) [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/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.11.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.11.plan
index fef1421..5d5f42b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.11.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.11.plan
@@ -1,50 +1,44 @@
-distribute result [$$29] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$31] [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 ([$$29]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$29] <- [{"age": $$34, "name": $$35}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$34, $$35]) [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 [$$32(ASC) ]  |PARTITIONED|
-            order (ASC, $$32) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$34, $$35, $$32]) [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|
-                    join (eq($$31, $$32)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HYBRID_HASH_JOIN [$$31][$$32]  |PARTITIONED|
-                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
-                        project ([$$34, $$31]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$34, $$31] <- [$$p1.getField("age"), $$p1.getField("id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- ASSIGN  |PARTITIONED|
+    assign [$$31] <- [{"age": $$36, "name": $$37}] project: [$$31] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$36, $$37]) [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 [$$34(ASC) ]  |PARTITIONED|
+          order (ASC, $$34) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              project ([$$36, $$37, $$34]) [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|
+                  join (eq($$33, $$34)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HYBRID_HASH_JOIN [$$33][$$34]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$33]  |PARTITIONED|
+                      assign [$$36, $$33] <- [$$p1.getField("age"), $$p1.getField("id")] project: [$$36, $$33] [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]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$p1] <- test.ParquetDataset [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|
-                              data-scan []<-[$$p1] <- test.ParquetDataset [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 [$$32]  |PARTITIONED|
-                        project ([$$35, $$32]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$35, $$32] <- [$$p2.getField("name"), $$p2.getField("id")] [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|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
+                      assign [$$37, $$34] <- [$$p2.getField("name"), $$p2.getField("id")] project: [$$37, $$34] [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]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$p2] <- test.ParquetDataset3 [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|
-                              data-scan []<-[$$p2] <- test.ParquetDataset3 [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|
+                              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/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.13.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.13.plan
index 5449c75..ec52c3d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.13.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.13.plan
@@ -1,50 +1,44 @@
-distribute result [$$29] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$31] [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 ([$$29]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$29] <- [{"age": $$34, "name": $$35}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$34, $$35]) [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 [$$32(ASC) ]  |PARTITIONED|
-            order (ASC, $$32) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$34, $$35, $$32]) [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|
-                    join (eq($$31, $$32)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HYBRID_HASH_JOIN [$$31][$$32]  |PARTITIONED|
-                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
-                        project ([$$34, $$31]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$34, $$31] <- [$$p1.getField("age"), $$p1.getField("id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- ASSIGN  |PARTITIONED|
+    assign [$$31] <- [{"age": $$36, "name": $$37}] project: [$$31] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$36, $$37]) [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 [$$34(ASC) ]  |PARTITIONED|
+          order (ASC, $$34) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              project ([$$36, $$37, $$34]) [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|
+                  join (eq($$33, $$34)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HYBRID_HASH_JOIN [$$33][$$34]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$33]  |PARTITIONED|
+                      assign [$$36, $$33] <- [$$p1.getField("age"), $$p1.getField("id")] project: [$$36, $$33] [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]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$p1] <- test.ParquetDataset project ({id:any,age:any}) [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|
-                              data-scan []<-[$$p1] <- test.ParquetDataset project ({id:any,age:any}) [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 [$$32]  |PARTITIONED|
-                        project ([$$35, $$32]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$35, $$32] <- [$$p2.getField("name"), $$p2.getField("id")] [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|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
+                      assign [$$37, $$34] <- [$$p2.getField("name"), $$p2.getField("id")] project: [$$37, $$34] [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]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$p2] <- test.ParquetDataset3 project ({name:any,id:any}) [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|
-                              data-scan []<-[$$p2] <- test.ParquetDataset3 project ({name:any,id:any}) [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|
+                              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/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.15.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.15.plan
index 51e4b81..b67932b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.15.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.15.plan
@@ -1,32 +1,26 @@
-distribute result [$$17] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$18] [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 ([$$17]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$17] <- [{"id": $$21, "name": $$22}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$21, $$22]) [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 [$$20(ASC) ]  |PARTITIONED|
-            order (ASC, $$20) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$21, $$22, $$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$22, $$21] <- [$$19.getField("name"), $$19.getField("id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ASSIGN  |PARTITIONED|
-                    project ([$$19, $$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$19, $$20] <- [$$p.getField("user"), $$p.getField("id")] [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]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$p] <- test.ParquetDataset4 [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|
+    assign [$$18] <- [{"id": $$22, "name": $$23}] project: [$$18] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      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]
+        -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+          order (ASC, $$21) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$23, $$22] <- [$$20.getField("name"), $$20.getField("id")] project: [$$22, $$23, $$21] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ASSIGN  |PARTITIONED|
+                assign [$$20, $$21] <- [$$p.getField("user"), $$p.getField("id")] project: [$$20, $$21] [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]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$p] <- test.ParquetDataset4 [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/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.17.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.17.plan
index 65a4217..9c357f9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.17.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.17.plan
@@ -1,32 +1,26 @@
-distribute result [$$17] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$18] [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 ([$$17]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$17] <- [{"id": $$21, "name": $$22}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$21, $$22]) [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 [$$20(ASC) ]  |PARTITIONED|
-            order (ASC, $$20) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$21, $$22, $$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$22, $$21] <- [$$19.getField("name"), $$19.getField("id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ASSIGN  |PARTITIONED|
-                    project ([$$19, $$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$19, $$20] <- [$$p.getField("user"), $$p.getField("id")] [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]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$p] <- test.ParquetDataset4 project ({id:any,user:{name:any,id:any}}) [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|
+    assign [$$18] <- [{"id": $$22, "name": $$23}] project: [$$18] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      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]
+        -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+          order (ASC, $$21) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$23, $$22] <- [$$20.getField("name"), $$20.getField("id")] project: [$$22, $$23, $$21] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ASSIGN  |PARTITIONED|
+                assign [$$20, $$21] <- [$$p.getField("user"), $$p.getField("id")] project: [$$20, $$21] [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]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$p] <- test.ParquetDataset4 project ({id:any,user:{name:any,id:any}}) [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/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.19.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.19.plan
index 67c9a9f..7abd8d0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.19.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.19.plan
@@ -1,12 +1,12 @@
-distribute result [$$31] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$33] [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|
-    aggregate [$$31] <- [agg-sql-sum($$32)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    aggregate [$$33] <- [agg-sql-sum($$34)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
     -- AGGREGATE  |UNPARTITIONED|
       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        aggregate [$$32] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        aggregate [$$34] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
         -- AGGREGATE  |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/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.21.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.21.plan
index 5229312..7a111cc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.21.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.21.plan
@@ -1,12 +1,12 @@
-distribute result [$$31] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$33] [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|
-    aggregate [$$31] <- [agg-sql-sum($$32)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    aggregate [$$33] <- [agg-sql-sum($$34)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
     -- AGGREGATE  |UNPARTITIONED|
       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        aggregate [$$32] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        aggregate [$$34] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
         -- AGGREGATE  |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/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.22.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.22.plan
index 65a4217..9c357f9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.22.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.22.plan
@@ -1,32 +1,26 @@
-distribute result [$$17] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$18] [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 ([$$17]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$17] <- [{"id": $$21, "name": $$22}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$21, $$22]) [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 [$$20(ASC) ]  |PARTITIONED|
-            order (ASC, $$20) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$21, $$22, $$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$22, $$21] <- [$$19.getField("name"), $$19.getField("id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ASSIGN  |PARTITIONED|
-                    project ([$$19, $$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$19, $$20] <- [$$p.getField("user"), $$p.getField("id")] [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]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$p] <- test.ParquetDataset4 project ({id:any,user:{name:any,id:any}}) [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|
+    assign [$$18] <- [{"id": $$22, "name": $$23}] project: [$$18] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      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]
+        -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+          order (ASC, $$21) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$23, $$22] <- [$$20.getField("name"), $$20.getField("id")] project: [$$22, $$23, $$21] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ASSIGN  |PARTITIONED|
+                assign [$$20, $$21] <- [$$p.getField("user"), $$p.getField("id")] project: [$$20, $$21] [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]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$p] <- test.ParquetDataset4 project ({id:any,user:{name:any,id:any}}) [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/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.03.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.03.plan
index 7269401..984665d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.03.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.03.plan
@@ -1,30 +1,26 @@
-distribute result [$$20] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$21] [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 ([$$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$20] <- [switch-case(true, is-array($$22), $$24, $$25)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$22, $$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]
-          -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
-            order (ASC, $$23) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                assign [$$24, $$25] <- [array-star($$22).getField("text"), $$22.getField("text")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$21] <- [switch-case(true, is-array($$23), $$25, $$26)] project: [$$21] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$23, $$25, $$26]) [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 [$$24(ASC) ]  |PARTITIONED|
+          order (ASC, $$24) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$25, $$26] <- [array-star($$23).getField("text"), $$23.getField("text")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ASSIGN  |PARTITIONED|
+                assign [$$23, $$24] <- [$$p.getField("arrayOrObject"), $$p.getField("id")] project: [$$23, $$24] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                 -- ASSIGN  |PARTITIONED|
-                  project ([$$22, $$23]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$22, $$23] <- [$$p.getField("arrayOrObject"), $$p.getField("id")] [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]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$p] <- test.ParquetDataset [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|
-                        data-scan []<-[$$p] <- test.ParquetDataset [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|
+                        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/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.05.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.05.plan
index cf67bc7..69729f2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.05.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.05.plan
@@ -1,30 +1,26 @@
-distribute result [$$20] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$21] [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 ([$$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$20] <- [switch-case(true, is-array($$22), $$24, $$25)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$22, $$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]
-          -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
-            order (ASC, $$23) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                assign [$$24, $$25] <- [array-star($$22).getField("text"), $$22.getField("text")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$21] <- [switch-case(true, is-array($$23), $$25, $$26)] project: [$$21] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$23, $$25, $$26]) [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 [$$24(ASC) ]  |PARTITIONED|
+          order (ASC, $$24) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$25, $$26] <- [array-star($$23).getField("text"), $$23.getField("text")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ASSIGN  |PARTITIONED|
+                assign [$$23, $$24] <- [$$p.getField("arrayOrObject"), $$p.getField("id")] project: [$$23, $$24] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                 -- ASSIGN  |PARTITIONED|
-                  project ([$$22, $$23]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$22, $$23] <- [$$p.getField("arrayOrObject"), $$p.getField("id")] [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]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$p] <- test.ParquetDataset project ({arrayOrObject:<[{text:any}],{text:any}>,id:any}) [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|
-                        data-scan []<-[$$p] <- test.ParquetDataset project ({arrayOrObject:<[{text:any}],{text:any}>,id:any}) [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|
+                        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/external-dataset/common/parquet/object-concat/object-concat.3.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.3.plan
index 2fe8496..d13773f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.3.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.3.plan
@@ -1,24 +1,22 @@
-distribute result [$$16] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$17] [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 ([$$16]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    project ([$$17]) [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 [$$18(ASC) ]  |PARTITIONED|
-        order (ASC, $$18) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$19(ASC) ]  |PARTITIONED|
+        order (ASC, $$19) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$16, $$18]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              assign [$$16, $$18] <- [object-concat($$p.getField("coordinates"), $$p.getField("user")).getField("name"), $$p.getField("id")] [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]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  data-scan []<-[$$p] <- test.ParquetDataset project ({coordinates:any,id:any,user:any}) [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|
+            assign [$$17, $$19] <- [object-concat($$p.getField("coordinates"), $$p.getField("user")).getField("name"), $$p.getField("id")] project: [$$17, $$19] [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]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                data-scan []<-[$$p] <- test.ParquetDataset project ({coordinates:any,id:any,user:any}) [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/external-dataset/common/parquet/object-concat/object-concat.5.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.5.plan
index 1e4c4e1..4d987be 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.5.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.5.plan
@@ -1,28 +1,24 @@
-distribute result [$$15] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$16] [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 ([$$15]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$15] <- [object-concat($$18, $$19)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$18, $$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 [$$17(ASC) ]  |PARTITIONED|
-            order (ASC, $$17) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$18, $$19, $$17]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$19, $$18, $$17] <- [$$p.getField("user"), $$p.getField("coordinates"), $$p.getField("id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ASSIGN  |PARTITIONED|
+    assign [$$16] <- [object-concat($$19, $$20)] project: [$$16] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$19, $$20]) [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 [$$18(ASC) ]  |PARTITIONED|
+          order (ASC, $$18) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$20, $$19, $$18] <- [$$p.getField("user"), $$p.getField("coordinates"), $$p.getField("id")] project: [$$19, $$20, $$18] [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]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$p] <- test.ParquetDataset project ({coordinates:any,id:any,user:any}) [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|
-                      data-scan []<-[$$p] <- test.ParquetDataset project ({coordinates:any,id:any,user:any}) [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|
+                      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/external-dataset/common/parquet/pushdown-plans/pushdown-plans.02.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.02.plan
index 7213e70..11e717e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.02.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.02.plan
@@ -1,50 +1,44 @@
-distribute result [$$48] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$51] [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|
-    project ([$$48]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$48] <- [{"$1": $$51}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$51] <- [agg-global-sql-sum($$53)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$53] <- [agg-local-sql-sum($$46)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              project ([$$46]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$51] <- [{"$1": $$54}] project: [$$51] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$54] <- [agg-global-sql-sum($$56)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$56] <- [agg-local-sql-sum($$49)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            assign [$$49] <- [object-length($$p1)] project: [$$49] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ASSIGN  |PARTITIONED|
+              project ([$$p1]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
               -- STREAM_PROJECT  |PARTITIONED|
-                assign [$$46] <- [object-length($$p1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ASSIGN  |PARTITIONED|
-                  project ([$$p1]) [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|
+                  join (eq($$52, $$53)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HYBRID_HASH_JOIN [$$53][$$52]  |PARTITIONED|
                     exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      join (eq($$49, $$50)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- HYBRID_HASH_JOIN [$$50][$$49]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                      assign [$$53] <- [$$p1.getField("id")] [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]
-                        -- HASH_PARTITION_EXCHANGE [$$50]  |PARTITIONED|
-                          assign [$$50] <- [$$p1.getField("id")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$p1] <- test.ParquetDataset1 [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|
-                              data-scan []<-[$$p1] <- test.ParquetDataset1 [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|
+                              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 [$$52]  |PARTITIONED|
+                      assign [$$52] <- [$$p2.getField("id")] project: [$$52] [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]
-                        -- HASH_PARTITION_EXCHANGE [$$49]  |PARTITIONED|
-                          project ([$$49]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            assign [$$49] <- [$$p2.getField("id")] [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]
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                data-scan []<-[$$p2] <- test.ParquetDataset2 project ({id:any}) [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|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$p2] <- test.ParquetDataset2 project ({id:any}) [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/external-dataset/common/parquet/pushdown-plans/pushdown-plans.04.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.04.plan
index 9ab9577..3993a4d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.04.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.04.plan
@@ -2,49 +2,43 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$69]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$69] <- [{"text": $$text, "$1": $$72}] [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]
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          group by ([$$text := $$80]) decor ([]) {
-                    aggregate [$$72] <- [agg-global-sql-sum($$79)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$80]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$80]  |PARTITIONED|
-              group by ([$$80 := $$71]) decor ([]) {
-                        aggregate [$$79] <- [agg-local-sql-sum(array-distinct($$74))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$71]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  select (eq(lowercase($$71), "string")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_SELECT  |PARTITIONED|
-                    project ([$$74, $$71]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$71] <- [$$ht.getField("text")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$69] <- [{"text": $$text, "$1": $$72}] project: [$$69] [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]
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        group by ([$$text := $$80]) decor ([]) {
+                  aggregate [$$72] <- [agg-global-sql-sum($$79)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$80]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$80]  |PARTITIONED|
+            group by ([$$80 := $$71]) decor ([]) {
+                      aggregate [$$79] <- [agg-local-sql-sum(array-distinct($$74))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$71]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                select (eq(lowercase($$71), "string")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STREAM_SELECT  |PARTITIONED|
+                  assign [$$71] <- [$$ht.getField("text")] project: [$$74, $$71] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ASSIGN  |PARTITIONED|
+                    unnest $$ht <- scan-collection($$74) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- UNNEST  |PARTITIONED|
+                      assign [$$74] <- [$$p1.getField("entities").getField("hashtags")] project: [$$74] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- ASSIGN  |PARTITIONED|
-                        unnest $$ht <- scan-collection($$74) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- UNNEST  |PARTITIONED|
-                          project ([$$74]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            assign [$$74] <- [$$p1.getField("entities").getField("hashtags")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- ASSIGN  |PARTITIONED|
-                              select (gt($$p1.getField("id"), 10)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_SELECT  |PARTITIONED|
-                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  data-scan []<-[$$p1] <- test.ParquetDataset1 project ({entities:{hashtags:any},id:any}) [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|
+                        select (gt($$p1.getField("id"), 10)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- STREAM_SELECT  |PARTITIONED|
+                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            data-scan []<-[$$p1] <- test.ParquetDataset1 project ({entities:{hashtags:any},id:any}) [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/external-dataset/common/parquet/pushdown-plans/pushdown-plans.05.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.05.plan
index 7a186a6..9b93e19 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.05.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.05.plan
@@ -2,51 +2,43 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$68]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$68] <- [{"text": $$text, "$1": $$71}] [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]
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          group by ([$$text := $$78]) decor ([]) {
-                    aggregate [$$71] <- [agg-global-sql-sum($$77)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$78]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
-              group by ([$$78 := $$70]) decor ([]) {
-                        aggregate [$$77] <- [agg-local-sql-sum(sql-sum($$74))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$70]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  select (eq(lowercase($$70), "string")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_SELECT  |PARTITIONED|
-                    project ([$$74, $$70]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$74, $$70] <- [$$ht.getField("indices"), $$ht.getField("text")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$68] <- [{"text": $$text, "$1": $$71}] project: [$$68] [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]
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        group by ([$$text := $$78]) decor ([]) {
+                  aggregate [$$71] <- [agg-global-sql-sum($$77)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$78]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
+            group by ([$$78 := $$70]) decor ([]) {
+                      aggregate [$$77] <- [agg-local-sql-sum(sql-sum($$74))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$70]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                select (eq(lowercase($$70), "string")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STREAM_SELECT  |PARTITIONED|
+                  assign [$$74, $$70] <- [$$ht.getField("indices"), $$ht.getField("text")] project: [$$74, $$70] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ASSIGN  |PARTITIONED|
+                    unnest $$ht <- scan-collection($$73) project: [$$ht] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- UNNEST  |PARTITIONED|
+                      assign [$$73] <- [$$p1.getField("entities").getField("hashtags")] project: [$$73] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- ASSIGN  |PARTITIONED|
-                        project ([$$ht]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          unnest $$ht <- scan-collection($$73) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- UNNEST  |PARTITIONED|
-                            project ([$$73]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              assign [$$73] <- [$$p1.getField("entities").getField("hashtags")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ASSIGN  |PARTITIONED|
-                                select (gt($$p1.getField("id"), 10)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    data-scan []<-[$$p1] <- test.ParquetDataset1 project ({entities:{hashtags:[{indices:any,text:any}]},id:any}) [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|
+                        select (gt($$p1.getField("id"), 10)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- STREAM_SELECT  |PARTITIONED|
+                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            data-scan []<-[$$p1] <- test.ParquetDataset1 project ({entities:{hashtags:[{indices:any,text:any}]},id:any}) [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/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.plan
index 06863c6..acb9726 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.plan
@@ -1,4 +1,4 @@
-distribute result [$$21] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$22] [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|
@@ -6,21 +6,17 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        project ([$$21]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          assign [$$21] <- [{"display_url": get-item($$22, 0).getField("display_url")}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- ASSIGN  |PARTITIONED|
-            limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_LIMIT  |PARTITIONED|
-              project ([$$22]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                assign [$$22] <- [$$p.getField("entities").getField("urls")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ASSIGN  |PARTITIONED|
+        assign [$$22] <- [{"display_url": get-item($$23, 0).getField("display_url")}] project: [$$22] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- ASSIGN  |PARTITIONED|
+          limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STREAM_LIMIT  |PARTITIONED|
+            assign [$$23] <- [$$p.getField("entities").getField("urls")] project: [$$23] [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]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                data-scan []<-[$$p] <- test.ParquetDataset1 condition (gt(sql-count($$p.getField("entities").getField("urls")), 10)) limit 10 project ({entities:{urls:[{display_url:any}]}}) [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|
-                    data-scan []<-[$$p] <- test.ParquetDataset1 condition (gt(sql-count($$p.getField("entities").getField("urls")), 10)) limit 10 project ({entities:{urls:[{display_url:any}]}}) [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|
+                    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/external-dataset/common/parquet/pushdown-plans/pushdown-plans.07.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.07.plan
index c3fc8f8..707548e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.07.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.07.plan
@@ -4,110 +4,94 @@
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$101]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$101] <- [{"uname": $$uname, "cnt": $$103}] [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 [$$103(DESC) ]  |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]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (topK: 10) (DESC, $$103) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STABLE_SORT [topK: 10] [$$103(DESC)]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    group by ([$$uname := $$114]) decor ([]) {
-                              aggregate [$$103] <- [agg-sql-sum($$113)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- AGGREGATE  |LOCAL|
-                                nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                           } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- SORT_GROUP_BY[$$114]  |PARTITIONED|
-                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- HASH_PARTITION_EXCHANGE [$$114]  |PARTITIONED|
-                        group by ([$$114 := $$102]) decor ([]) {
-                                  aggregate [$$113] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- AGGREGATE  |LOCAL|
-                                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- SORT_GROUP_BY[$$102]  |PARTITIONED|
-                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            project ([$$102]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      assign [$$101] <- [{"uname": $$uname, "cnt": $$103}] project: [$$101] [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 [$$103(DESC) ]  |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]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              order (topK: 10) (DESC, $$103) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STABLE_SORT [topK: 10] [$$103(DESC)]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  group by ([$$uname := $$114]) decor ([]) {
+                            aggregate [$$103] <- [agg-sql-sum($$113)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- AGGREGATE  |LOCAL|
+                              nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                         } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- SORT_GROUP_BY[$$114]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$114]  |PARTITIONED|
+                      group by ([$$114 := $$102]) decor ([]) {
+                                aggregate [$$113] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- AGGREGATE  |LOCAL|
+                                  nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                             } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- SORT_GROUP_BY[$$102]  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          select ($$92) project: [$$102] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- STREAM_SELECT  |PARTITIONED|
+                            project ([$$92, $$102]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- STREAM_PROJECT  |PARTITIONED|
-                              select ($$92) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_SELECT  |PARTITIONED|
-                                project ([$$92, $$102]) [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|
+                                group by ([$$112 := $$110]) decor ([$$102]) {
+                                          aggregate [$$92] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- AGGREGATE  |LOCAL|
+                                            select (not(is-missing($$111))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                            -- STREAM_SELECT  |LOCAL|
+                                              nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- PRE_CLUSTERED_GROUP_BY[$$110]  |PARTITIONED|
                                   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    group by ([$$112 := $$110]) decor ([$$102]) {
-                                              aggregate [$$92] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- AGGREGATE  |LOCAL|
-                                                select (not(is-missing($$111))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- STREAM_SELECT  |LOCAL|
-                                                  nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                           } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- PRE_CLUSTERED_GROUP_BY[$$110]  |PARTITIONED|
+                                    order (ASC, $$110) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- STABLE_SORT [$$110(ASC)]  |PARTITIONED|
                                       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        order (ASC, $$110) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- STABLE_SORT [$$110(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$110]  |PARTITIONED|
+                                        project ([$$102, $$111, $$110]) [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]
-                                          -- HASH_PARTITION_EXCHANGE [$$110]  |PARTITIONED|
-                                            project ([$$102, $$111, $$110]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            left outer join (eq($$104, $$88)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                            -- HYBRID_HASH_JOIN [$$104][$$88]  |PARTITIONED|
                                               exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                left outer join (eq($$104, $$88)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- HYBRID_HASH_JOIN [$$104][$$88]  |PARTITIONED|
-                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- HASH_PARTITION_EXCHANGE [$$104]  |PARTITIONED|
-                                                    running-aggregate [$$110] <- [create-query-uid()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                      project ([$$102, $$104]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        assign [$$104] <- [$$ht1.getField("text")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          project ([$$102, $$ht1]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            unnest $$ht1 <- scan-collection($$105) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                            -- UNNEST  |PARTITIONED|
-                                                              project ([$$105, $$102]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                assign [$$105, $$102] <- [$$p1.getField("entities").getField("hashtags"), $$p1.getField("user").getField("name")] [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]
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    data-scan []<-[$$p1] <- test.ParquetDataset1 project ({entities:{hashtags:[{text:any}]},user:{name:any}}) [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 [$$88]  |PARTITIONED|
-                                                    project ([$$111, $$88]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      assign [$$111, $$88] <- [true, $$ht2.getField("text")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- HASH_PARTITION_EXCHANGE [$$104]  |PARTITIONED|
+                                                running-aggregate [$$110] <- [create-query-uid()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                  assign [$$104] <- [$$ht1.getField("text")] project: [$$102, $$104] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    unnest $$ht1 <- scan-collection($$105) project: [$$102, $$ht1] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                    -- UNNEST  |PARTITIONED|
+                                                      assign [$$105, $$102] <- [$$p1.getField("entities").getField("hashtags"), $$p1.getField("user").getField("name")] project: [$$105, $$102] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                                       -- ASSIGN  |PARTITIONED|
-                                                        project ([$$ht2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          unnest $$ht2 <- scan-collection($$106) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                          -- UNNEST  |PARTITIONED|
-                                                            project ([$$106]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              assign [$$106] <- [$$p2.getField("entities").getField("hashtags")] [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]
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  data-scan []<-[$$p2] <- test.ParquetDataset2 project ({entities:{hashtags:[{text:any}]}}) [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  |PARTITIONED|
+                                                          data-scan []<-[$$p1] <- test.ParquetDataset1 project ({entities:{hashtags:[{text:any}]},user:{name:any}}) [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 [$$88]  |PARTITIONED|
+                                                assign [$$111, $$88] <- [true, $$ht2.getField("text")] project: [$$111, $$88] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                -- ASSIGN  |PARTITIONED|
+                                                  unnest $$ht2 <- scan-collection($$106) project: [$$ht2] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                  -- UNNEST  |PARTITIONED|
+                                                    assign [$$106] <- [$$p2.getField("entities").getField("hashtags")] project: [$$106] [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]
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        data-scan []<-[$$p2] <- test.ParquetDataset2 project ({entities:{hashtags:[{text:any}]}}) [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/external-dataset/common/parquet/pushdown-plans/pushdown-plans.08.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.08.plan
index 94921f4..cccf9a3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.08.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.08.plan
@@ -2,26 +2,37 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$65]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$65] <- [{"$1": $$66}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$66] <- [agg-sql-sum($$75)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$75] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              select (or($$53, $$57)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$53, $$57]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
+    assign [$$65] <- [{"$1": $$66}] project: [$$65] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$66] <- [agg-sql-sum($$75)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$75] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            select (or($$53, $$57)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$53, $$57]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_PROJECT  |PARTITIONED|
+                subplan {
+                          aggregate [$$57] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- AGGREGATE  |LOCAL|
+                            select (eq($$70, 2)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- STREAM_SELECT  |LOCAL|
+                              assign [$$70] <- [$$e.getField("x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- ASSIGN  |LOCAL|
+                                unnest $$e <- scan-collection($$67) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- SUBPLAN  |PARTITIONED|
                   subplan {
-                            aggregate [$$57] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            aggregate [$$53] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- AGGREGATE  |LOCAL|
-                              select (eq($$70, 2)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              select (eq($$68, 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                               -- STREAM_SELECT  |LOCAL|
-                                assign [$$70] <- [$$e.getField("x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                assign [$$68] <- [$$e.getField("x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                 -- ASSIGN  |LOCAL|
                                   unnest $$e <- scan-collection($$67) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                   -- UNNEST  |LOCAL|
@@ -29,28 +40,13 @@
                                     -- NESTED_TUPLE_SOURCE  |LOCAL|
                          } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- SUBPLAN  |PARTITIONED|
-                    subplan {
-                              aggregate [$$53] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- AGGREGATE  |LOCAL|
-                                select (eq($$68, 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |LOCAL|
-                                  assign [$$68] <- [$$e.getField("x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ASSIGN  |LOCAL|
-                                    unnest $$e <- scan-collection($$67) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- SUBPLAN  |PARTITIONED|
-                      project ([$$67]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$67] <- [$$p.getField("val1")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ASSIGN  |PARTITIONED|
+                    assign [$$67] <- [$$p.getField("val1")] project: [$$67] [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]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        data-scan []<-[$$p] <- test.ParquetDataset1 project ({val1:[{x:any}]}) [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|
-                            data-scan []<-[$$p] <- test.ParquetDataset1 project ({val1:[{x:any}]}) [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|
+                            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/external-dataset/common/parquet/pushdown-plans/pushdown-plans.09.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.09.plan
index 1de2926..cf265bf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.09.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.09.plan
@@ -2,26 +2,37 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$65]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$65] <- [{"$1": $$66}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$66] <- [agg-sql-sum($$75)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$75] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              select (or($$53, $$57)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$53, $$57]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
+    assign [$$65] <- [{"$1": $$66}] project: [$$65] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$66] <- [agg-sql-sum($$75)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$75] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            select (or($$53, $$57)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$53, $$57]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_PROJECT  |PARTITIONED|
+                subplan {
+                          aggregate [$$57] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- AGGREGATE  |LOCAL|
+                            select (eq($$70, 2)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- STREAM_SELECT  |LOCAL|
+                              assign [$$70] <- [$$e.getField("y")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- ASSIGN  |LOCAL|
+                                unnest $$e <- scan-collection($$67) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- SUBPLAN  |PARTITIONED|
                   subplan {
-                            aggregate [$$57] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            aggregate [$$53] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- AGGREGATE  |LOCAL|
-                              select (eq($$70, 2)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              select (eq($$68, 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                               -- STREAM_SELECT  |LOCAL|
-                                assign [$$70] <- [$$e.getField("y")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                assign [$$68] <- [$$e.getField("x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                 -- ASSIGN  |LOCAL|
                                   unnest $$e <- scan-collection($$67) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                   -- UNNEST  |LOCAL|
@@ -29,28 +40,13 @@
                                     -- NESTED_TUPLE_SOURCE  |LOCAL|
                          } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- SUBPLAN  |PARTITIONED|
-                    subplan {
-                              aggregate [$$53] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- AGGREGATE  |LOCAL|
-                                select (eq($$68, 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |LOCAL|
-                                  assign [$$68] <- [$$e.getField("x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ASSIGN  |LOCAL|
-                                    unnest $$e <- scan-collection($$67) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- SUBPLAN  |PARTITIONED|
-                      project ([$$67]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$67] <- [$$p.getField("val1")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ASSIGN  |PARTITIONED|
+                    assign [$$67] <- [$$p.getField("val1")] project: [$$67] [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]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        data-scan []<-[$$p] <- test.ParquetDataset1 project ({val1:[{x:any,y:any}]}) [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|
-                            data-scan []<-[$$p] <- test.ParquetDataset1 project ({val1:[{x:any,y:any}]}) [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|
+                            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/external-dataset/common/parquet/pushdown-plans/pushdown-plans.10.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.10.plan
index b7ac895..3199625 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.10.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.10.plan
@@ -2,57 +2,53 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$65]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$65] <- [{"$1": $$66}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$66] <- [agg-sql-sum($$75)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$75] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              select (or($$53, $$57)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$53, $$57]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  subplan {
-                            aggregate [$$57] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- AGGREGATE  |LOCAL|
-                              select (eq($$70, 2)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_SELECT  |LOCAL|
-                                assign [$$70] <- [$$e.getField("y")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- ASSIGN  |LOCAL|
-                                  unnest $$e <- scan-collection($$69) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- SUBPLAN  |PARTITIONED|
-                    project ([$$69, $$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      subplan {
-                                aggregate [$$53] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- AGGREGATE  |LOCAL|
-                                  select (eq($$68, 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- STREAM_SELECT  |LOCAL|
-                                    assign [$$68] <- [$$e.getField("x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- ASSIGN  |LOCAL|
-                                      unnest $$e <- scan-collection($$67) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- SUBPLAN  |PARTITIONED|
-                        project ([$$69, $$67]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$69, $$67] <- [$$p.getField("val2"), $$p.getField("val1")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- ASSIGN  |PARTITIONED|
+    assign [$$65] <- [{"$1": $$66}] project: [$$65] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$66] <- [agg-sql-sum($$75)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$75] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            select (or($$53, $$57)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$53, $$57]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_PROJECT  |PARTITIONED|
+                subplan {
+                          aggregate [$$57] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- AGGREGATE  |LOCAL|
+                            select (eq($$70, 2)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- STREAM_SELECT  |LOCAL|
+                              assign [$$70] <- [$$e.getField("y")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- ASSIGN  |LOCAL|
+                                unnest $$e <- scan-collection($$69) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- SUBPLAN  |PARTITIONED|
+                  project ([$$69, $$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    subplan {
+                              aggregate [$$53] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- AGGREGATE  |LOCAL|
+                                select (eq($$68, 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- STREAM_SELECT  |LOCAL|
+                                  assign [$$68] <- [$$e.getField("x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- ASSIGN  |LOCAL|
+                                    unnest $$e <- scan-collection($$67) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- SUBPLAN  |PARTITIONED|
+                      assign [$$69, $$67] <- [$$p.getField("val2"), $$p.getField("val1")] project: [$$69, $$67] [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]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$p] <- test.ParquetDataset1 project ({val2:[{y:any}],val1:[{x:any}]}) [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|
-                              data-scan []<-[$$p] <- test.ParquetDataset1 project ({val2:[{y:any}],val1:[{x:any}]}) [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|
+                              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/external-dataset/common/parquet/pushdown-plans/pushdown-plans.11.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.11.plan
index bb6814a..85643e8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.11.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.11.plan
@@ -2,21 +2,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$31]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$31] <- [{"$1": get-item(get-item(get-item($$32, 0).getField("f2"), 0), 0), "$2": get-item($$46.getField("f3"), 1), "$3": get-item(get-item($$46.getField("f2"), 1), 1)}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$31] <- [{"$1": get-item(get-item(get-item($$32, 0).getField("f2"), 0), 0), "$2": get-item($$46.getField("f3"), 1), "$3": get-item(get-item($$46.getField("f2"), 1), 1)}] project: [$$31] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      assign [$$46] <- [get-item($$32, 1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- ASSIGN  |PARTITIONED|
-        assign [$$46] <- [get-item($$32, 1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        assign [$$32] <- [$$p.getField("f1")] project: [$$32] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
         -- ASSIGN  |PARTITIONED|
-          project ([$$32]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STREAM_PROJECT  |PARTITIONED|
-            assign [$$32] <- [$$p.getField("f1")] [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]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            data-scan []<-[$$p] <- test.ParquetDataset1 project ({f1:[{f2:[[any]],f3:[any]}]}) [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|
-                data-scan []<-[$$p] <- test.ParquetDataset1 project ({f1:[{f2:[[any]],f3:[any]}]}) [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|
+                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/external-dataset/common/parquet/pushdown-plans/pushdown-plans.12.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.12.plan
index 8b117a9..3081703 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.12.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.12.plan
@@ -2,35 +2,27 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$31]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$31] <- [{"$1": get-item(get-item($$35, 0), 0), "$2": get-item($$36, 1), "$3": get-item(get-item($$37, 1), 1)}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$35, $$36, $$37]) [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 [$$34(ASC) ]  |PARTITIONED|
-            order (ASC, $$34) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$35, $$36, $$37, $$34]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$37, $$36] <- [$$48.getField("f2"), $$48.getField("f3")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$31] <- [{"$1": get-item(get-item($$35, 0), 0), "$2": get-item($$36, 1), "$3": get-item(get-item($$37, 1), 1)}] project: [$$31] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$35, $$36, $$37]) [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 [$$34(ASC) ]  |PARTITIONED|
+          order (ASC, $$34) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$37, $$36] <- [$$48.getField("f2"), $$48.getField("f3")] project: [$$35, $$36, $$37, $$34] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ASSIGN  |PARTITIONED|
+                assign [$$48, $$35] <- [get-item($$33, 1), get-item($$33, 0).getField("f2")] project: [$$34, $$48, $$35] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  assign [$$33, $$34] <- [$$p.getField("f1"), $$p.getField("x")] project: [$$33, $$34] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- ASSIGN  |PARTITIONED|
-                    project ([$$34, $$48, $$35]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$48, $$35] <- [get-item($$33, 1), get-item($$33, 0).getField("f2")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
-                        project ([$$33, $$34]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$33, $$34] <- [$$p.getField("f1"), $$p.getField("x")] [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]
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              data-scan []<-[$$p] <- test.ParquetDataset1 project ({x:any,f1:[{f2:[[any]],f3:[any]}]}) [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  |PARTITIONED|
+                      data-scan []<-[$$p] <- test.ParquetDataset1 project ({x:any,f1:[{f2:[[any]],f3:[any]}]}) [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/external-dataset/common/parquet/pushdown-plans/pushdown-plans.13.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.13.plan
index 3266da3..40d1206 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.13.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.13.plan
@@ -2,19 +2,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$32]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$32] <- [{"$1": get-item(get-item(get-item($$33, 0).getField("f2"), 0), 0), "$2": get-item($$48.getField("f3"), 1), "$3": get-item(get-item($$48.getField("f2"), 1), 1), "x": $$p.getField("x")}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$32] <- [{"$1": get-item(get-item(get-item($$33, 0).getField("f2"), 0), 0), "$2": get-item($$48.getField("f3"), 1), "$3": get-item(get-item($$48.getField("f2"), 1), 1), "x": $$p.getField("x")}] project: [$$32] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      assign [$$48] <- [get-item($$33, 1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- ASSIGN  |PARTITIONED|
-        assign [$$48] <- [get-item($$33, 1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        assign [$$33] <- [$$p.getField("f1")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
         -- ASSIGN  |PARTITIONED|
-          assign [$$33] <- [$$p.getField("f1")] [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]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              data-scan []<-[$$p] <- test.ParquetDataset1 project ({x:any,f1:[{f2:[[any]],f3:[any]}]}) [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  |PARTITIONED|
+            data-scan []<-[$$p] <- test.ParquetDataset1 project ({x:any,f1:[{f2:[[any]],f3:[any]}]}) [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/external-dataset/common/parquet/pushdown-plans/pushdown-plans.14.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.14.plan
index 2b10297..ed92bc7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.14.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.14.plan
@@ -2,35 +2,27 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$32]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$32] <- [{"$1": get-item(get-item($$36, 0), 0), "$2": get-item($$37, 1), "$3": get-item(get-item($$38, 1), 1), "x": $$39}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$36, $$37, $$38, $$39]) [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 [$$35(ASC) ]  |PARTITIONED|
-            order (ASC, $$35) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$36, $$37, $$38, $$39, $$35]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$38, $$37] <- [$$50.getField("f2"), $$50.getField("f3")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$32] <- [{"$1": get-item(get-item($$36, 0), 0), "$2": get-item($$37, 1), "$3": get-item(get-item($$38, 1), 1), "x": $$39}] project: [$$32] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$36, $$37, $$38, $$39]) [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 [$$35(ASC) ]  |PARTITIONED|
+          order (ASC, $$35) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$38, $$37] <- [$$50.getField("f2"), $$50.getField("f3")] project: [$$36, $$37, $$38, $$39, $$35] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ASSIGN  |PARTITIONED|
+                assign [$$50, $$36] <- [get-item($$34, 1), get-item($$34, 0).getField("f2")] project: [$$35, $$39, $$50, $$36] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  assign [$$34, $$35, $$39] <- [$$p.getField("f1"), $$p.getField("y"), $$p.getField("x")] project: [$$34, $$35, $$39] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- ASSIGN  |PARTITIONED|
-                    project ([$$35, $$39, $$50, $$36]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$50, $$36] <- [get-item($$34, 1), get-item($$34, 0).getField("f2")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
-                        project ([$$34, $$35, $$39]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$34, $$35, $$39] <- [$$p.getField("f1"), $$p.getField("y"), $$p.getField("x")] [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]
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              data-scan []<-[$$p] <- test.ParquetDataset1 project ({x:any,y:any,f1:[{f2:[[any]],f3:[any]}]}) [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  |PARTITIONED|
+                      data-scan []<-[$$p] <- test.ParquetDataset1 project ({x:any,y:any,f1:[{f2:[[any]],f3:[any]}]}) [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/external-dataset/common/parquet/pushdown-plans/pushdown-plans.15.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.15.plan
index c575b1b..89c67d1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.15.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.15.plan
@@ -2,23 +2,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$35]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$35] <- [{"$1": get-item(get-item(get-item($$36, 0).getField("f2"), 0), 0), "$2": get-item($$42, 1), "$3": get-item(get-item($$54.getField("f2"), 1), 1), "f4": $$42.getField("f4")}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$35] <- [{"$1": get-item(get-item(get-item($$36, 0).getField("f2"), 0), 0), "$2": get-item($$42, 1), "$3": get-item(get-item($$54.getField("f2"), 1), 1), "f4": $$42.getField("f4")}] project: [$$35] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      assign [$$42] <- [$$54.getField("f3")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- ASSIGN  |PARTITIONED|
-        assign [$$42] <- [$$54.getField("f3")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        assign [$$54] <- [get-item($$36, 1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
         -- ASSIGN  |PARTITIONED|
-          assign [$$54] <- [get-item($$36, 1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          assign [$$36] <- [$$p.getField("f1")] project: [$$36] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- ASSIGN  |PARTITIONED|
-            project ([$$36]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              assign [$$36] <- [$$p.getField("f1")] [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]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$p] <- test.ParquetDataset1 project ({f1:[{f2:[[any]],f3:<[any],{f4:any}>}]}) [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|
-                  data-scan []<-[$$p] <- test.ParquetDataset1 project ({f1:[{f2:[[any]],f3:<[any],{f4:any}>}]}) [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|
+                  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/external-dataset/common/query-with-limit-plan/result.001.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/query-with-limit-plan/result.001.plan
index 196c0ce..3b473af 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/query-with-limit-plan/result.001.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/query-with-limit-plan/result.001.plan
@@ -1,4 +1,4 @@
-distribute result [$$13] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$14] [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|
@@ -6,17 +6,15 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        project ([$$13]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          assign [$$13] <- [{"test": $$test}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- ASSIGN  |PARTITIONED|
-            limit 3 [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]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                data-scan []<-[$$test] <- test.test limit 3 [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|
+        assign [$$14] <- [{"test": $$test}] project: [$$14] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- ASSIGN  |PARTITIONED|
+          limit 3 [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]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$test] <- test.test limit 3 [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/geojson/datatype/result.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.03.adm
index 2a16ab1..06c68c8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.03.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.03.adm
@@ -1,14 +1,13 @@
-{ "Geometries": { "id": 123, "myGeometry": {"type":"Point","coordinates":[-118.4,33.93],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
-{ "Geometries": { "id": 124, "myGeometry": {"type":"Polygon","coordinates":[[[8.7599721,49.7103028],[8.759997,49.7102752],[8.7600145,49.7102818],[8.7600762,49.7102133],[8.760178,49.7102516],[8.7600914,49.7103478],[8.7599721,49.7103028]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
-{ "Geometries": { "id": 126, "myGeometry": {"type":"LineString","coordinates":[[-69.1991349,-12.6006222],[-69.199136,-12.599842],[-69.1982979,-12.5998268],[-69.1982598,-12.599869],[-69.1982188,-12.5998698],[-69.19817,-12.5998707],[-69.198125,-12.5998218],[-69.1973024,-12.5998133],[-69.1972972,-12.6003109],[-69.197394,-12.6003514],[-69.1973906,-12.6009231],[-69.1975115,-12.601026],[-69.1975081,-12.6010968]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
-{ "Geometries": { "id": 127, "myGeometry": {"type":"MultiPoint","coordinates":[[10,40],[40,30],[20,20],[30,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
-{ "Geometries": { "id": 128, "myGeometry": {"type":"MultiLineString","coordinates":[[[10,10],[20,20],[10,40]],[[40,40],[30,30],[40,20],[30,10]]],"crs":null} } }
-{ "Geometries": { "id": 129, "myGeometry": {"type":"MultiPolygon","coordinates":[[[[40,40],[20,45],[45,30],[40,40]]],[[[20,35],[10,30],[10,10],[30,5],[45,20],[20,35]],[[30,20],[20,15],[20,25],[30,20]]]],"crs":null} } }
-{ "Geometries": { "id": 130, "myGeometry": {"type":"Point","coordinates":[-71.1043443253471,42.3150676015829],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
-{ "Geometries": { "id": 131, "myGeometry": {"type":"Point","coordinates":[1,2,3],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
-{ "Geometries": { "id": 132, "myGeometry": {"type":"Point","coordinates":[1,2,3,4],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
-{ "Geometries": { "id": 133, "myGeometry": {"type":"Polygon","coordinates":[[[743238,2967416],[743265.625,2967416],[743265,2967450],[743238,2967450],[743238,2967416]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
-{ "Geometries": { "id": 134, "myGeometry": {"type":"LineString","coordinates":[[-113.98,39.198],[-113.981,39.195]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
-{ "Geometries": { "id": 135, "myGeometry": {"type":"LineString","coordinates":[[1,2],[4,5],[7,8]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
-{ "Geometries": { "id": 136, "myGeometry": {"type":"Polygon","coordinates":[[[10,10],[11,10],[11,11],[10,11],[10,10]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
-{ "Geometries": { "id": 137, "myGeometry": {"type":"Polygon","coordinates":[[[35,10],[45,45],[15,40],[10,20],[35,10]],[[20,30],[35,35],[30,20],[20,30]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 123, "myGeometry": {"type":"Point","coordinates":[-118.4,33.93]} } }
+{ "Geometries": { "id": 124, "myGeometry": {"type":"Polygon","coordinates":[[[8.7599721,49.7103028],[8.759997,49.7102752],[8.7600145,49.7102818],[8.7600762,49.7102133],[8.760178,49.7102516],[8.7600914,49.7103478],[8.7599721,49.7103028]]]} } }
+{ "Geometries": { "id": 126, "myGeometry": {"type":"LineString","coordinates":[[-69.1991349,-12.6006222],[-69.199136,-12.599842],[-69.1982979,-12.5998268],[-69.1982598,-12.599869],[-69.1982188,-12.5998698],[-69.19817,-12.5998707],[-69.198125,-12.5998218],[-69.1973024,-12.5998133],[-69.1972972,-12.6003109],[-69.197394,-12.6003514],[-69.1973906,-12.6009231],[-69.1975115,-12.601026],[-69.1975081,-12.6010968]]} } }
+{ "Geometries": { "id": 127, "myGeometry": {"type":"MultiPoint","coordinates":[[10,40],[40,30],[20,20],[30,10]]} } }
+{ "Geometries": { "id": 128, "myGeometry": {"type":"MultiLineString","coordinates":[[[10,10],[20,20],[10,40]],[[40,40],[30,30],[40,20],[30,10]]]} } }
+{ "Geometries": { "id": 129, "myGeometry": {"type":"MultiPolygon","coordinates":[[[[40,40],[20,45],[45,30],[40,40]]],[[[20,35],[10,30],[10,10],[30,5],[45,20],[20,35]],[[30,20],[20,15],[20,25],[30,20]]]]} } }
+{ "Geometries": { "id": 130, "myGeometry": {"type":"Point","coordinates":[-71.1043443253471,42.3150676015829]} } }
+{ "Geometries": { "id": 131, "myGeometry": {"type":"Point","coordinates":[1,2,3]} } }
+{ "Geometries": { "id": 133, "myGeometry": {"type":"Polygon","coordinates":[[[743238,2967416],[743238,2967450],[743265,2967450],[743265.625,2967416],[743238,2967416]]]} } }
+{ "Geometries": { "id": 134, "myGeometry": {"type":"LineString","coordinates":[[-113.98,39.198],[-113.981,39.195]]} } }
+{ "Geometries": { "id": 135, "myGeometry": {"type":"LineString","coordinates":[[1,2],[4,5],[7,8]]} } }
+{ "Geometries": { "id": 136, "myGeometry": {"type":"Polygon","coordinates":[[[10,10],[10,11],[11,11],[11,10],[10,10]]]} } }
+{ "Geometries": { "id": 137, "myGeometry": {"type":"Polygon","coordinates":[[[35,10],[45,45],[15,40],[10,20],[35,10]],[[20,30],[35,35],[30,20],[20,30]]]} } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.04.adm
index 6cba06a..688fcf1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.04.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.04.adm
@@ -1,14 +1,13 @@
-{ "Type": "Point", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 0, "NPoints": 1, "XMax": -118.4, "XMin": -118.4, "YMax": 33.93, "YMin": 33.93, "Binary": hex("01010000009A99999999995DC0D7A3703D0AF74040"), "GeoJSON": "{\"type\":\"Point\",\"coordinates\":[-118.4,33.93],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POINT (-118.4 33.93)" }
-{ "Type": "Polygon", "Area": 1.3755215000294761E-8, "Coordinate dimension": 2, "Dimension": 2, "NPoints": 6, "XMax": 8.760178, "XMin": 8.7599721, "YMax": 49.7103478, "YMin": 49.7102133, "Binary": hex("01030000000100000007000000B1BE26101B852140ED20C033EBDA4840C11DA8531E8521407694394CEADA4840BAF8DB9E20852140D5F89683EADA484098EF2AB5288521409557F844E8DA48400247020D3685214041F74086E9DA484014F131B32A8521408DE43CADECDA4840B1BE26101B852140ED20C033EBDA4840"), "GeoJSON": "{\"type\":\"Polygon\",\"coordinates\":[[[8.7599721,49.7103028],[8.759997,49.7102752],[8.7600145,49.7102818],[8.7600762,49.7102133],[8.760178,49.7102516],[8.7600914,49.7103478],[8.7599721,49.7103028]]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POLYGON ((8.7599721 49.7103028, 8.759997 49.7102752, 8.7600145 49.7102818, 8.7600762 49.7102133, 8.760178 49.7102516, 8.7600914 49.7103478, 8.7599721 49.7103028))" }
-{ "Type": "LineString", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 1, "NPoints": 13, "XMax": -69.1972972, "XMin": -69.199136, "YMax": -12.5998133, "YMin": -12.6010968, "Binary": hex("01020000000D00000081BF4EA0BE4C51C0CE80C4C0843329C033DDEBA4BE4C51C0D388997D1E3329C064D7ACE9B04C51C05787927F1C3329C02347DF49B04C51C0B9C49107223329C0F4DCE79DAF4C51C054B76922223329C0184339D1AE4C51C042A89C40223329C048E17A14AE4C51C010DBCCD71B3329C007DB3E9AA04C51C0636996BA1A3329C0E9656F84A04C51C05ADD45F35B3329C04E9B711AA24C51C033373A42613329C06CE22E0CA24C51C066D24B31AC3329C05F454607A44C51C065170CAEB93329C07C8C03F9A34C51C0E475B3F5C23329C0"), "GeoJSON": "{\"type\":\"LineString\",\"coordinates\":[[-69.1991349,-12.6006222],[-69.199136,-12.599842],[-69.1982979,-12.5998268],[-69.1982598,-12.599869],[-69.1982188,-12.5998698],[-69.19817,-12.5998707],[-69.198125,-12.5998218],[-69.1973024,-12.5998133],[-69.1972972,-12.6003109],[-69.197394,-12.6003514],[-69.1973906,-12.6009231],[-69.1975115,-12.601026],[-69.1975081,-12.6010968]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "LINESTRING (-69.1991349 -12.6006222, -69.199136 -12.599842, -69.1982979 -12.5998268, -69.1982598 -12.599869, -69.1982188 -12.5998698, -69.19817 -12.5998707, -69.198125 -12.5998218, -69.1973024 -12.5998133, -69.1972972 -12.6003109, -69.197394 -12.6003514, -69.1973906 -12.6009231, -69.1975115 -12.601026, -69.1975081 -12.6010968)" }
-{ "Type": "MultiPoint", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 0, "NPoints": 4, "XMax": 40.0, "XMin": 10.0, "YMax": 40.0, "YMin": 10.0, "Binary": hex("010400000004000000010100000000000000000024400000000000004440010100000000000000000044400000000000003E4001010000000000000000003440000000000000344001010000000000000000003E400000000000002440"), "GeoJSON": "{\"type\":\"MultiPoint\",\"coordinates\":[[10,40],[40,30],[20,20],[30,10]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "MULTIPOINT ((10 40), (40 30), (20 20), (30 10))" }
-{ "Type": "MultiLineString", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 1, "NPoints": 7, "XMax": 40.0, "XMin": 10.0, "YMax": 40.0, "YMin": 10.0, "Binary": hex("010500000002000000010200000003000000000000000000244000000000000024400000000000003440000000000000344000000000000024400000000000004440010200000004000000000000000000444000000000000044400000000000003E400000000000003E40000000000000444000000000000034400000000000003E400000000000002440"), "GeoJSON": "{\"type\":\"MultiLineString\",\"coordinates\":[[[10,10],[20,20],[10,40]],[[40,40],[30,30],[40,20],[30,10]]],\"crs\":null}", "WKT": "MULTILINESTRING ((10 10, 20 20, 10 40), (40 40, 30 30, 40 20, 30 10))" }
-{ "Type": "MultiPolygon", "Area": 712.5, "Coordinate dimension": 2, "Dimension": 2, "NPoints": 11, "XMax": 45.0, "XMin": 10.0, "YMax": 45.0, "YMin": 5.0, "Binary": hex("01060000000200000001030000000100000004000000000000000000444000000000000044400000000000003440000000000080464000000000008046400000000000003E4000000000000044400000000000004440010300000002000000060000000000000000003440000000000080414000000000000024400000000000003E40000000000000244000000000000024400000000000003E4000000000000014400000000000804640000000000000344000000000000034400000000000804140040000000000000000003E40000000000000344000000000000034400000000000002E40000000000000344000000000000039400000000000003E400000000000003440"), "GeoJSON": "{\"type\":\"MultiPolygon\",\"coordinates\":[[[[40,40],[20,45],[45,30],[40,40]]],[[[20,35],[10,30],[10,10],[30,5],[45,20],[20,35]],[[30,20],[20,15],[20,25],[30,20]]]],\"crs\":null}", "WKT": "MULTIPOLYGON (((40 40, 20 45, 45 30, 40 40)), ((20 35, 10 30, 10 10, 30 5, 45 20, 20 35), (30 20, 20 15, 20 25, 30 20)))" }
-{ "Type": "Point", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 0, "NPoints": 1, "XMax": -71.1043443253471, "XMin": -71.1043443253471, "YMax": 42.3150676015829, "YMin": 42.3150676015829, "Binary": hex("0101000000E538D293ADC651C0F3699A2254284540"), "GeoJSON": "{\"type\":\"Point\",\"coordinates\":[-71.1043443253471,42.3150676015829],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POINT (-71.1043443253471 42.3150676015829)" }
-{ "Type": "Point", "Area": 0.0, "Coordinate dimension": 3, "Dimension": 0, "NPoints": 1, "XMax": 1.0, "XMin": 1.0, "YMax": 2.0, "YMin": 2.0, "Binary": hex("01E9030000000000000000F03F00000000000000400000000000000840"), "GeoJSON": "{\"type\":\"Point\",\"coordinates\":[1,2,3],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POINT Z (1 2 3)" }
-{ "Type": "Point", "Area": 0.0, "Coordinate dimension": 4, "Dimension": 0, "NPoints": 1, "XMax": 1.0, "XMin": 1.0, "YMax": 2.0, "YMin": 2.0, "Binary": hex("01B90B0000000000000000F03F000000000000004000000000000008400000000000001040"), "GeoJSON": "{\"type\":\"Point\",\"coordinates\":[1,2,3,4],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POINT ZM (1 2 3 4)" }
-{ "Type": "Polygon", "Area": 928.625, "Coordinate dimension": 2, "Dimension": 2, "NPoints": 4, "XMax": 743265.625, "XMin": 743238.0, "YMax": 2967450.0, "YMin": 2967416.0, "Binary": hex("01030000000100000005000000000000008CAE264100000000BCA3464100000040C3AE264100000000BCA3464100000000C2AE264100000000CDA34641000000008CAE264100000000CDA34641000000008CAE264100000000BCA34641"), "GeoJSON": "{\"type\":\"Polygon\",\"coordinates\":[[[743238,2967416],[743265.625,2967416],[743265,2967450],[743238,2967450],[743238,2967416]]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POLYGON ((743238 2967416, 743265.625 2967416, 743265 2967450, 743238 2967450, 743238 2967416))" }
-{ "Type": "LineString", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 1, "NPoints": 2, "XMax": -113.98, "XMin": -113.981, "YMax": 39.198, "YMin": 39.195, "Binary": hex("0102000000020000001F85EB51B87E5CC0D34D621058994340105839B4C87E5CC0295C8FC2F5984340"), "GeoJSON": "{\"type\":\"LineString\",\"coordinates\":[[-113.98,39.198],[-113.981,39.195]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "LINESTRING (-113.98 39.198, -113.981 39.195)" }
-{ "Type": "LineString", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 1, "NPoints": 3, "XMax": 7.0, "XMin": 1.0, "YMax": 8.0, "YMin": 2.0, "Binary": hex("010200000003000000000000000000F03F0000000000000040000000000000104000000000000014400000000000001C400000000000002040"), "GeoJSON": "{\"type\":\"LineString\",\"coordinates\":[[1,2],[4,5],[7,8]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "LINESTRING (1 2, 4 5, 7 8)" }
-{ "Type": "Polygon", "Area": 1.0, "Coordinate dimension": 2, "Dimension": 2, "NPoints": 4, "XMax": 11.0, "XMin": 10.0, "YMax": 11.0, "YMin": 10.0, "Binary": hex("010300000001000000050000000000000000002440000000000000244000000000000026400000000000002440000000000000264000000000000026400000000000002440000000000000264000000000000024400000000000002440"), "GeoJSON": "{\"type\":\"Polygon\",\"coordinates\":[[[10,10],[11,10],[11,11],[10,11],[10,10]]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POLYGON ((10 10, 11 10, 11 11, 10 11, 10 10))" }
-{ "Type": "Polygon", "Area": 675.0, "Coordinate dimension": 2, "Dimension": 2, "NPoints": 7, "XMax": 45.0, "XMin": 10.0, "YMax": 45.0, "YMin": 10.0, "Binary": hex("0103000000020000000500000000000000008041400000000000002440000000000080464000000000008046400000000000002E40000000000000444000000000000024400000000000003440000000000080414000000000000024400400000000000000000034400000000000003E40000000000080414000000000008041400000000000003E40000000000000344000000000000034400000000000003E40"), "GeoJSON": "{\"type\":\"Polygon\",\"coordinates\":[[[35,10],[45,45],[15,40],[10,20],[35,10]],[[20,30],[35,35],[30,20],[20,30]]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POLYGON ((35 10, 45 45, 15 40, 10 20, 35 10), (20 30, 35 35, 30 20, 20 30))" }
+{ "Type": "Point", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 0, "NPoints": 1, "XMax": -118.4, "XMin": -118.4, "YMax": 33.93, "YMin": 33.93, "Binary": hex("01010000009A99999999995DC0D7A3703D0AF74040"), "GeoJSON": "{\"type\":\"Point\",\"coordinates\":[-118.4,33.93]}", "WKT": "POINT (-118.4 33.93)" }
+{ "Type": "Polygon", "Area": 1.3755215000294761E-8, "Coordinate dimension": 2, "Dimension": 2, "NPoints": 6, "XMax": 8.760178, "XMin": 8.7599721, "YMax": 49.7103478, "YMin": 49.7102133, "Binary": hex("01030000000100000007000000B1BE26101B852140ED20C033EBDA4840C11DA8531E8521407694394CEADA4840BAF8DB9E20852140D5F89683EADA484098EF2AB5288521409557F844E8DA48400247020D3685214041F74086E9DA484014F131B32A8521408DE43CADECDA4840B1BE26101B852140ED20C033EBDA4840"), "GeoJSON": "{\"type\":\"Polygon\",\"coordinates\":[[[8.7599721,49.7103028],[8.759997,49.7102752],[8.7600145,49.7102818],[8.7600762,49.7102133],[8.760178,49.7102516],[8.7600914,49.7103478],[8.7599721,49.7103028]]]}", "WKT": "POLYGON ((8.7599721 49.7103028, 8.759997 49.7102752, 8.7600145 49.7102818, 8.7600762 49.7102133, 8.760178 49.7102516, 8.7600914 49.7103478, 8.7599721 49.7103028))" }
+{ "Type": "LineString", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 1, "NPoints": 13, "XMax": -69.1972972, "XMin": -69.199136, "YMax": -12.5998133, "YMin": -12.6010968, "Binary": hex("01020000000D00000081BF4EA0BE4C51C0CE80C4C0843329C033DDEBA4BE4C51C0D388997D1E3329C064D7ACE9B04C51C05787927F1C3329C02347DF49B04C51C0B9C49107223329C0F4DCE79DAF4C51C054B76922223329C0184339D1AE4C51C042A89C40223329C048E17A14AE4C51C010DBCCD71B3329C007DB3E9AA04C51C0636996BA1A3329C0E9656F84A04C51C05ADD45F35B3329C04E9B711AA24C51C033373A42613329C06CE22E0CA24C51C066D24B31AC3329C05F454607A44C51C065170CAEB93329C07C8C03F9A34C51C0E475B3F5C23329C0"), "GeoJSON": "{\"type\":\"LineString\",\"coordinates\":[[-69.1991349,-12.6006222],[-69.199136,-12.599842],[-69.1982979,-12.5998268],[-69.1982598,-12.599869],[-69.1982188,-12.5998698],[-69.19817,-12.5998707],[-69.198125,-12.5998218],[-69.1973024,-12.5998133],[-69.1972972,-12.6003109],[-69.197394,-12.6003514],[-69.1973906,-12.6009231],[-69.1975115,-12.601026],[-69.1975081,-12.6010968]]}", "WKT": "LINESTRING (-69.1991349 -12.6006222, -69.199136 -12.599842, -69.1982979 -12.5998268, -69.1982598 -12.599869, -69.1982188 -12.5998698, -69.19817 -12.5998707, -69.198125 -12.5998218, -69.1973024 -12.5998133, -69.1972972 -12.6003109, -69.197394 -12.6003514, -69.1973906 -12.6009231, -69.1975115 -12.601026, -69.1975081 -12.6010968)" }
+{ "Type": "MultiPoint", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 0, "NPoints": 4, "XMax": 40.0, "XMin": 10.0, "YMax": 40.0, "YMin": 10.0, "Binary": hex("010400000004000000010100000000000000000024400000000000004440010100000000000000000044400000000000003E4001010000000000000000003440000000000000344001010000000000000000003E400000000000002440"), "GeoJSON": "{\"type\":\"MultiPoint\",\"coordinates\":[[10,40],[40,30],[20,20],[30,10]]}", "WKT": "MULTIPOINT ((10 40), (40 30), (20 20), (30 10))" }
+{ "Type": "MultiLineString", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 1, "NPoints": 7, "XMax": 40.0, "XMin": 10.0, "YMax": 40.0, "YMin": 10.0, "Binary": hex("010500000002000000010200000003000000000000000000244000000000000024400000000000003440000000000000344000000000000024400000000000004440010200000004000000000000000000444000000000000044400000000000003E400000000000003E40000000000000444000000000000034400000000000003E400000000000002440"), "GeoJSON": "{\"type\":\"MultiLineString\",\"coordinates\":[[[10,10],[20,20],[10,40]],[[40,40],[30,30],[40,20],[30,10]]]}", "WKT": "MULTILINESTRING ((10 10, 20 20, 10 40), (40 40, 30 30, 40 20, 30 10))" }
+{ "Type": "MultiPolygon", "Area": 712.5, "Coordinate dimension": 2, "Dimension": 2, "NPoints": 11, "XMax": 45.0, "XMin": 10.0, "YMax": 45.0, "YMin": 5.0, "Binary": hex("01060000000200000001030000000100000004000000000000000000444000000000000044400000000000003440000000000080464000000000008046400000000000003E4000000000000044400000000000004440010300000002000000060000000000000000003440000000000080414000000000000024400000000000003E40000000000000244000000000000024400000000000003E4000000000000014400000000000804640000000000000344000000000000034400000000000804140040000000000000000003E40000000000000344000000000000034400000000000002E40000000000000344000000000000039400000000000003E400000000000003440"), "GeoJSON": "{\"type\":\"MultiPolygon\",\"coordinates\":[[[[40,40],[20,45],[45,30],[40,40]]],[[[20,35],[10,30],[10,10],[30,5],[45,20],[20,35]],[[30,20],[20,15],[20,25],[30,20]]]]}", "WKT": "MULTIPOLYGON (((40 40, 20 45, 45 30, 40 40)), ((20 35, 10 30, 10 10, 30 5, 45 20, 20 35), (30 20, 20 15, 20 25, 30 20)))" }
+{ "Type": "Point", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 0, "NPoints": 1, "XMax": -71.1043443253471, "XMin": -71.1043443253471, "YMax": 42.3150676015829, "YMin": 42.3150676015829, "Binary": hex("0101000000E538D293ADC651C0F3699A2254284540"), "GeoJSON": "{\"type\":\"Point\",\"coordinates\":[-71.1043443253471,42.3150676015829]}", "WKT": "POINT (-71.1043443253471 42.3150676015829)" }
+{ "Type": "Point", "Area": 0.0, "Coordinate dimension": 3, "Dimension": 0, "NPoints": 1, "XMax": 1.0, "XMin": 1.0, "YMax": 2.0, "YMin": 2.0, "Binary": hex("0101000080000000000000F03F00000000000000400000000000000840"), "GeoJSON": "{\"type\":\"Point\",\"coordinates\":[1,2,3]}", "WKT": "POINT Z(1 2 3)" }
+{ "Type": "Polygon", "Area": 928.625, "Coordinate dimension": 2, "Dimension": 2, "NPoints": 4, "XMax": 743265.625, "XMin": 743238.0, "YMax": 2967450.0, "YMin": 2967416.0, "Binary": hex("01030000000100000005000000000000008CAE264100000000BCA34641000000008CAE264100000000CDA3464100000000C2AE264100000000CDA3464100000040C3AE264100000000BCA34641000000008CAE264100000000BCA34641"), "GeoJSON": "{\"type\":\"Polygon\",\"coordinates\":[[[743238,2967416],[743238,2967450],[743265,2967450],[743265.625,2967416],[743238,2967416]]]}", "WKT": "POLYGON ((743238 2967416, 743238 2967450, 743265 2967450, 743265.625 2967416, 743238 2967416))" }
+{ "Type": "LineString", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 1, "NPoints": 2, "XMax": -113.98, "XMin": -113.981, "YMax": 39.198, "YMin": 39.195, "Binary": hex("0102000000020000001F85EB51B87E5CC0D34D621058994340105839B4C87E5CC0295C8FC2F5984340"), "GeoJSON": "{\"type\":\"LineString\",\"coordinates\":[[-113.98,39.198],[-113.981,39.195]]}", "WKT": "LINESTRING (-113.98 39.198, -113.981 39.195)" }
+{ "Type": "LineString", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 1, "NPoints": 3, "XMax": 7.0, "XMin": 1.0, "YMax": 8.0, "YMin": 2.0, "Binary": hex("010200000003000000000000000000F03F0000000000000040000000000000104000000000000014400000000000001C400000000000002040"), "GeoJSON": "{\"type\":\"LineString\",\"coordinates\":[[1,2],[4,5],[7,8]]}", "WKT": "LINESTRING (1 2, 4 5, 7 8)" }
+{ "Type": "Polygon", "Area": 1.0, "Coordinate dimension": 2, "Dimension": 2, "NPoints": 4, "XMax": 11.0, "XMin": 10.0, "YMax": 11.0, "YMin": 10.0, "Binary": hex("010300000001000000050000000000000000002440000000000000244000000000000024400000000000002640000000000000264000000000000026400000000000002640000000000000244000000000000024400000000000002440"), "GeoJSON": "{\"type\":\"Polygon\",\"coordinates\":[[[10,10],[10,11],[11,11],[11,10],[10,10]]]}", "WKT": "POLYGON ((10 10, 10 11, 11 11, 11 10, 10 10))" }
+{ "Type": "Polygon", "Area": 675.0, "Coordinate dimension": 2, "Dimension": 2, "NPoints": 7, "XMax": 45.0, "XMin": 10.0, "YMax": 45.0, "YMin": 10.0, "Binary": hex("0103000000020000000500000000000000008041400000000000002440000000000080464000000000008046400000000000002E40000000000000444000000000000024400000000000003440000000000080414000000000000024400400000000000000000034400000000000003E40000000000080414000000000008041400000000000003E40000000000000344000000000000034400000000000003E40"), "GeoJSON": "{\"type\":\"Polygon\",\"coordinates\":[[[35,10],[45,45],[15,40],[10,20],[35,10]],[[20,30],[35,35],[30,20],[20,30]]]}", "WKT": "POLYGON ((35 10, 45 45, 15 40, 10 20, 35 10), (20 30, 35 35, 30 20, 20 30))" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.05.adm
index 0815bfa..5ea6fd5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.05.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.05.adm
@@ -1,4 +1,3 @@
-{ "X": -118.4, "Y": 33.93, "Z": 0.0, "M": NaN }
-{ "X": -71.1043443253471, "Y": 42.3150676015829, "Z": 0.0, "M": NaN }
-{ "X": 1.0, "Y": 2.0, "Z": 3.0, "M": NaN }
-{ "X": 1.0, "Y": 2.0, "Z": 3.0, "M": 4.0 }
\ No newline at end of file
+{ "X": -118.4, "Y": 33.93, "Z": NaN, "M": NaN }
+{ "X": -71.1043443253471, "Y": 42.3150676015829, "Z": NaN, "M": NaN }
+{ "X": 1.0, "Y": 2.0, "Z": 3.0, "M": NaN }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.06.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.06.adm
index 044e31a..39ced5e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.06.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.06.adm
@@ -1,4 +1,4 @@
-{ "NumInteriorRings": 0, "ExteriorRing": {"type":"LineString","coordinates":[[8.7599721,49.7103028],[8.759997,49.7102752],[8.7600145,49.7102818],[8.7600762,49.7102133],[8.760178,49.7102516],[8.7600914,49.7103478],[8.7599721,49.7103028]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
-{ "NumInteriorRings": 0, "ExteriorRing": {"type":"LineString","coordinates":[[743238,2967416],[743265.625,2967416],[743265,2967450],[743238,2967450],[743238,2967416]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
-{ "NumInteriorRings": 0, "ExteriorRing": {"type":"LineString","coordinates":[[10,10],[11,10],[11,11],[10,11],[10,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
-{ "NumInteriorRings": 1, "ExteriorRing": {"type":"LineString","coordinates":[[35,10],[45,45],[15,40],[10,20],[35,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
\ No newline at end of file
+{ "NumInteriorRings": 0, "ExteriorRing": {"type":"LineString","coordinates":[[8.7599721,49.7103028],[8.759997,49.7102752],[8.7600145,49.7102818],[8.7600762,49.7102133],[8.760178,49.7102516],[8.7600914,49.7103478],[8.7599721,49.7103028]]} }
+{ "NumInteriorRings": 0, "ExteriorRing": {"type":"LineString","coordinates":[[743238,2967416],[743238,2967450],[743265,2967450],[743265.625,2967416],[743238,2967416]]} }
+{ "NumInteriorRings": 0, "ExteriorRing": {"type":"LineString","coordinates":[[10,10],[10,11],[11,11],[11,10],[10,10]]} }
+{ "NumInteriorRings": 1, "ExteriorRing": {"type":"LineString","coordinates":[[35,10],[45,45],[15,40],[10,20],[35,10]]} }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.07.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.07.adm
index 7967722..3a026d5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.07.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.07.adm
@@ -1,4 +1,4 @@
-{ "Length": 0.004058119099397876, "Boundary": {"type":"MultiPoint","coordinates":[[-69.1991349,-12.6006222],[-69.1975081,-12.6010968]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
-{ "Length": 78.9292222699217, "Boundary": {"type":"MultiPoint","coordinates":[[10,10],[10,40],[40,40],[30,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
-{ "Length": 0.0031622776601655037, "Boundary": {"type":"MultiPoint","coordinates":[[-113.98,39.198],[-113.981,39.195]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
-{ "Length": 8.48528137423857, "Boundary": {"type":"MultiPoint","coordinates":[[1,2],[7,8]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Length": 0.004058119099397876, "Boundary": {"type":"MultiPoint","coordinates":[[-69.1991349,-12.6006222],[-69.1975081,-12.6010968]]} }
+{ "Length": 78.9292222699217, "Boundary": {"type":"MultiPoint","coordinates":[[10,10],[10,40],[30,10],[40,40]]} }
+{ "Length": 0.0031622776601655037, "Boundary": {"type":"MultiPoint","coordinates":[[-113.98,39.198],[-113.981,39.195]]} }
+{ "Length": 8.48528137423857, "Boundary": {"type":"MultiPoint","coordinates":[[1,2],[7,8]]} }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.08.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.08.adm
index c0d2341..1987970 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.08.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.08.adm
@@ -1,3 +1,3 @@
-{ "PointN": {"type":"Point","coordinates":[-69.199136,-12.599842],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "StartPoint": {"type":"Point","coordinates":[-69.1991349,-12.6006222],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Envelope": {"type":"Polygon","coordinates":[[[-69.199136,-12.6010968],[-69.1972972,-12.6010968],[-69.1972972,-12.5998133],[-69.199136,-12.5998133],[-69.199136,-12.6010968]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
-{ "PointN": {"type":"Point","coordinates":[-69.199136,-12.599842],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "StartPoint": {"type":"Point","coordinates":[-113.98,39.198],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Envelope": {"type":"Polygon","coordinates":[[[-113.981,39.195],[-113.98,39.195],[-113.98,39.198],[-113.981,39.198],[-113.981,39.195]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
-{ "PointN": {"type":"Point","coordinates":[-69.199136,-12.599842],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "StartPoint": {"type":"Point","coordinates":[1,2],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Envelope": {"type":"Polygon","coordinates":[[[1,2],[7,2],[7,8],[1,8],[1,2]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
\ No newline at end of file
+{ "PointN": {"type":"Point","coordinates":[-69.199136,-12.599842]}, "StartPoint": {"type":"Point","coordinates":[-69.1991349,-12.6006222]}, "Envelope": {"type":"Polygon","coordinates":[[[-69.199136,-12.6010968],[-69.199136,-12.5998133],[-69.1972972,-12.5998133],[-69.1972972,-12.6010968],[-69.199136,-12.6010968]]]} }
+{ "PointN": {"type":"Point","coordinates":[-69.199136,-12.599842]}, "StartPoint": {"type":"Point","coordinates":[-113.98,39.198]}, "Envelope": {"type":"Polygon","coordinates":[[[-113.981,39.195],[-113.981,39.198],[-113.98,39.198],[-113.98,39.195],[-113.981,39.195]]]} }
+{ "PointN": {"type":"Point","coordinates":[-69.199136,-12.599842]}, "StartPoint": {"type":"Point","coordinates":[1,2]}, "Envelope": {"type":"Polygon","coordinates":[[[1,2],[1,8],[7,8],[7,2],[1,2]]]} }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.09.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.09.adm
index 893f2dc..8a42de2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.09.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.09.adm
@@ -7,7 +7,6 @@
 { "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
 { "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
 { "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
-{ "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
 { "IsClosed": false, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
 { "IsClosed": false, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
 { "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/index/result.20.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/index/result.20.adm
index ea141c9..bebded7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/index/result.20.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/index/result.20.adm
@@ -1,14 +1,13 @@
-{ "Geometries": { "id": 123, "myGeometry": {"type":"Point","coordinates":[-118.4,33.93],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
-{ "Geometries": { "id": 124, "myGeometry": {"type":"Polygon","coordinates":[[[8.7599721,49.7103028],[8.759997,49.7102752],[8.7600145,49.7102818],[8.7600762,49.7102133],[8.760178,49.7102516],[8.7600914,49.7103478],[8.7599721,49.7103028]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
-{ "Geometries": { "id": 126, "myGeometry": {"type":"LineString","coordinates":[[-69.1991349,-12.6006222],[-69.199136,-12.599842],[-69.1982979,-12.5998268],[-69.1982598,-12.599869],[-69.1982188,-12.5998698],[-69.19817,-12.5998707],[-69.198125,-12.5998218],[-69.1973024,-12.5998133],[-69.1972972,-12.6003109],[-69.197394,-12.6003514],[-69.1973906,-12.6009231],[-69.1975115,-12.601026],[-69.1975081,-12.6010968]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
-{ "Geometries": { "id": 127, "myGeometry": {"type":"MultiPoint","coordinates":[[10,40],[40,30],[20,20],[30,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
-{ "Geometries": { "id": 128, "myGeometry": {"type":"MultiLineString","coordinates":[[[10,10],[20,20],[10,40]],[[40,40],[30,30],[40,20],[30,10]]],"crs":null} } }
-{ "Geometries": { "id": 129, "myGeometry": {"type":"MultiPolygon","coordinates":[[[[40,40],[20,45],[45,30],[40,40]]],[[[20,35],[10,30],[10,10],[30,5],[45,20],[20,35]],[[30,20],[20,15],[20,25],[30,20]]]],"crs":null} } }
-{ "Geometries": { "id": 130, "myGeometry": {"type":"Point","coordinates":[-71.1043443253471,42.3150676015829],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
-{ "Geometries": { "id": 131, "myGeometry": {"type":"Point","coordinates":[1,2,3],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
-{ "Geometries": { "id": 132, "myGeometry": {"type":"Point","coordinates":[1,2,3,4],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
-{ "Geometries": { "id": 133, "myGeometry": {"type":"Polygon","coordinates":[[[743238,2967416],[743265.625,2967416],[743265,2967450],[743238,2967450],[743238,2967416]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
-{ "Geometries": { "id": 134, "myGeometry": {"type":"LineString","coordinates":[[-113.98,39.198],[-113.981,39.195]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
-{ "Geometries": { "id": 135, "myGeometry": {"type":"LineString","coordinates":[[1,2],[4,5],[7,8]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
-{ "Geometries": { "id": 136, "myGeometry": {"type":"Polygon","coordinates":[[[10,10],[11,10],[11,11],[10,11],[10,10]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
-{ "Geometries": { "id": 137, "myGeometry": {"type":"Polygon","coordinates":[[[35,10],[45,45],[15,40],[10,20],[35,10]],[[20,30],[35,35],[30,20],[20,30]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
\ No newline at end of file
+{ "Geometries": { "id": 123, "myGeometry": {"type":"Point","coordinates":[-118.4,33.93]} } }
+{ "Geometries": { "id": 124, "myGeometry": {"type":"Polygon","coordinates":[[[8.7599721,49.7103028],[8.759997,49.7102752],[8.7600145,49.7102818],[8.7600762,49.7102133],[8.760178,49.7102516],[8.7600914,49.7103478],[8.7599721,49.7103028]]]} } }
+{ "Geometries": { "id": 126, "myGeometry": {"type":"LineString","coordinates":[[-69.1991349,-12.6006222],[-69.199136,-12.599842],[-69.1982979,-12.5998268],[-69.1982598,-12.599869],[-69.1982188,-12.5998698],[-69.19817,-12.5998707],[-69.198125,-12.5998218],[-69.1973024,-12.5998133],[-69.1972972,-12.6003109],[-69.197394,-12.6003514],[-69.1973906,-12.6009231],[-69.1975115,-12.601026],[-69.1975081,-12.6010968]]} } }
+{ "Geometries": { "id": 127, "myGeometry": {"type":"MultiPoint","coordinates":[[10,40],[40,30],[20,20],[30,10]]} } }
+{ "Geometries": { "id": 128, "myGeometry": {"type":"MultiLineString","coordinates":[[[10,10],[20,20],[10,40]],[[40,40],[30,30],[40,20],[30,10]]]} } }
+{ "Geometries": { "id": 129, "myGeometry": {"type":"MultiPolygon","coordinates":[[[[40,40],[20,45],[45,30],[40,40]]],[[[20,35],[10,30],[10,10],[30,5],[45,20],[20,35]],[[30,20],[20,15],[20,25],[30,20]]]]} } }
+{ "Geometries": { "id": 130, "myGeometry": {"type":"Point","coordinates":[-71.1043443253471,42.3150676015829]} } }
+{ "Geometries": { "id": 131, "myGeometry": {"type":"Point","coordinates":[1,2,3]} } }
+{ "Geometries": { "id": 133, "myGeometry": {"type":"Polygon","coordinates":[[[743238,2967416],[743238,2967450],[743265,2967450],[743265.625,2967416],[743238,2967416]]]} } }
+{ "Geometries": { "id": 134, "myGeometry": {"type":"LineString","coordinates":[[-113.98,39.198],[-113.981,39.195]]} } }
+{ "Geometries": { "id": 135, "myGeometry": {"type":"LineString","coordinates":[[1,2],[4,5],[7,8]]} } }
+{ "Geometries": { "id": 136, "myGeometry": {"type":"Polygon","coordinates":[[[10,10],[10,11],[11,11],[11,10],[10,10]]]} } }
+{ "Geometries": { "id": 137, "myGeometry": {"type":"Polygon","coordinates":[[[35,10],[45,45],[15,40],[10,20],[35,10]],[[20,30],[35,35],[30,20],[20,30]]]} } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/index/result.21.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/index/result.21.adm
index 2ceb6e0..7628d62 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/index/result.21.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/index/result.21.adm
@@ -1,3 +1,2 @@
 131
-132
 135
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.10.adm
index a1e16dd..bfa005e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.10.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.10.adm
@@ -1 +1 @@
-{"type":"Polygon","coordinates":[[[-7,4.2],[-7.1,5],[-7.1,4.3],[-7,4.2]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
\ No newline at end of file
+{"type":"Polygon","coordinates":[[[-7,4.2],[-7.1,5],[-7.1,4.3],[-7,4.2]]]}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.15.adm
new file mode 100644
index 0000000..14ec637
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.15.adm
@@ -0,0 +1 @@
+{ "a": NaN, "b": NaN, "c": 0.0, "d": 0.0, "e": 3.0, "f": 1.0, "g": -1.0, "h": -3.0, "i": 1.0, "j": -1.0, "k": 2.0, "l": 0.0, "m": 0.0, "n": -1.0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.04.adm
index e353336..57d5885 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.04.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.04.adm
@@ -1,10 +1,10 @@
-{ "Union": {"type":"LineString","coordinates":[[0,0],[1,1],[0,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"Point","coordinates":[0,2],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"LineString","coordinates":[[0,0],[1,1],[0,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
-{ "Union": {"type":"MultiLineString","coordinates":[[[1,1],[2,2]],[[0,2],[1,2],[2,2]],[[2,2],[3,3]],[[3,3],[4,2],[5,2]],[[3,3],[4,4],[5,5],[6,6]]],"crs":null}, "Intersection": {"type":"LineString","coordinates":[[2,2],[3,3]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"MultiLineString","coordinates":[[[0,2],[1,2],[2,2],[1,1]],[[5,2],[4,2],[3,3],[4,4],[5,5],[6,6]]],"crs":null} }
-{ "Union": {"type":"LineString","coordinates":[[0,0],[0,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"Point","coordinates":[0,0],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"LineString","coordinates":[[0,0],[0,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
-{ "Union": {"type":"LineString","coordinates":[[1,1],[-1,-1],[2,3.5],[1,3],[1,2],[2,1]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"Point","coordinates":[-1,-1],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"LineString","coordinates":[[1,1],[-1,-1],[2,3.5],[1,3],[1,2],[2,1]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
-{ "Union": {"type":"MultiLineString","coordinates":[[[1,1],[2,2]],[[0,2],[1,2],[2,2]],[[2,2],[3,2],[4,2],[5,2]],[[2,2],[3,3],[4,4],[5,5],[6,6]]],"crs":null}, "Intersection": {"type":"Point","coordinates":[2,2],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"MultiLineString","coordinates":[[[1,1],[2,2]],[[0,2],[1,2],[2,2]],[[2,2],[3,2],[4,2],[5,2]],[[2,2],[3,3],[4,4],[5,5],[6,6]]],"crs":null} }
-{ "Union": {"type":"LineString","coordinates":[[1,1],[2,2],[3,3],[4,4],[5,5],[6,6]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"MultiPolygon","coordinates":[],"crs":null}, "SymDifference": {"type":"LineString","coordinates":[[1,1],[2,2],[3,3],[4,4],[5,5],[6,6]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
-{ "Union": {"type":"LineString","coordinates":[[0,0],[5,5],[10,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"LineString","coordinates":[[0,0],[5,5],[10,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"MultiLineString","coordinates":[],"crs":null} }
-{ "Union": {"type":"MultiLineString","coordinates":[[[1,2],[3,4]],[[5,6],[7,8]]],"crs":null}, "Intersection": {"type":"MultiPolygon","coordinates":[],"crs":null}, "SymDifference": {"type":"MultiLineString","coordinates":[[[1,2],[3,4]],[[5,6],[7,8]]],"crs":null} }
-{ "Union": {"type":"LineString","coordinates":[[0,0],[1,1],[0,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"Point","coordinates":[0,2],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"LineString","coordinates":[[0,0],[1,1],[0,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
-{ "Union": {"type":"Polygon","coordinates":[[[0,0],[1,0],[1,1],[0,1],[0,0]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"Polygon","coordinates":[[[0.25,0.25],[0.5,0.25],[0.5,0.5],[0.25,0.5],[0.25,0.25]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"Polygon","coordinates":[[[0,0],[1,0],[1,1],[0,1],[0,0]],[[0.25,0.25],[0.25,0.5],[0.5,0.5],[0.5,0.25],[0.25,0.25]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Union": {"type":"LineString","coordinates":[[0,0],[1,1],[0,2]]}, "Intersection": {"type":"Point","coordinates":[0,2]}, "SymDifference": {"type":"LineString","coordinates":[[0,0],[1,1],[0,2]]} }
+{ "Union": {"type":"MultiLineString","coordinates":[[[1,1],[2,2]],[[2,2],[3,3]],[[3,3],[4,4],[5,5],[6,6]],[[0,2],[1,2],[2,2]],[[3,3],[4,2],[5,2]]]}, "Intersection": {"type":"LineString","coordinates":[[2,2],[3,3]]}, "SymDifference": {"type":"MultiLineString","coordinates":[[[1,1],[2,2]],[[3,3],[4,4],[5,5],[6,6]],[[0,2],[1,2],[2,2]],[[3,3],[4,2],[5,2]]]} }
+{ "Union": {"type":"LineString","coordinates":[[0,0],[0,2]]}, "Intersection": {"type":"Point","coordinates":[0,0]}, "SymDifference": {"type":"LineString","coordinates":[[0,0],[0,2]]} }
+{ "Union": {"type":"MultiLineString","coordinates":[[[1,1],[-1,-1],[1,2]],[[1,2],[2,3.5],[1,3],[1,2]],[[1,2],[2,1]]]}, "Intersection": {"type":"Point","coordinates":[-1,-1]}, "SymDifference": {"type":"MultiLineString","coordinates":[[[1,1],[-1,-1],[1,2]],[[1,2],[2,3.5],[1,3],[1,2]],[[1,2],[2,1]]]} }
+{ "Union": {"type":"MultiLineString","coordinates":[[[1,1],[2,2]],[[2,2],[3,3],[4,4],[5,5],[6,6]],[[0,2],[1,2],[2,2]],[[2,2],[3,2],[4,2],[5,2]]]}, "Intersection": {"type":"Point","coordinates":[2,2]}, "SymDifference": {"type":"MultiLineString","coordinates":[[[1,1],[2,2]],[[2,2],[3,3],[4,4],[5,5],[6,6]],[[0,2],[1,2],[2,2]],[[2,2],[3,2],[4,2],[5,2]]]} }
+{ "Union": {"type":"GeometryCollection","geometries":[{"type":"Point","coordinates":[0,0]},{"type":"LineString","coordinates":[[1,1],[2,2],[3,3],[4,4],[5,5],[6,6]]}]}, "Intersection": , "SymDifference": {"type":"GeometryCollection","geometries":[{"type":"Point","coordinates":[0,0]},{"type":"LineString","coordinates":[[1,1],[2,2],[3,3],[4,4],[5,5],[6,6]]}]} }
+{ "Union": {"type":"MultiLineString","coordinates":[[[0,0],[5,5]],[[5,5],[10,10]]]}, "Intersection": {"type":"MultiLineString","coordinates":[[[0,0],[5,5]],[[5,5],[10,10]]]}, "SymDifference": {"type":"LineString","coordinates":[]} }
+{ "Union": {"type":"MultiLineString","coordinates":[[[1,2],[3,4]],[[5,6],[7,8]]]}, "Intersection": {"type":"LineString","coordinates":[]}, "SymDifference": {"type":"MultiLineString","coordinates":[[[1,2],[3,4]],[[5,6],[7,8]]]} }
+{ "Union": {"type":"LineString","coordinates":[[0,0],[1,1],[0,2]]}, "Intersection": {"type":"Point","coordinates":[0,2]}, "SymDifference": {"type":"LineString","coordinates":[[0,0],[1,1],[0,2]]} }
+{ "Union": {"type":"Polygon","coordinates":[[[0,0],[0,1],[1,1],[1,0],[0,0]]]}, "Intersection": {"type":"Polygon","coordinates":[[[0.25,0.25],[0.25,0.5],[0.5,0.5],[0.5,0.25],[0.25,0.25]]]}, "SymDifference": {"type":"Polygon","coordinates":[[[0,0],[0,1],[1,1],[1,0],[0,0]],[[0.25,0.25],[0.5,0.25],[0.5,0.5],[0.25,0.5],[0.25,0.25]]]} }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.05.adm
index 5123e09..d7e9872 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.05.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.05.adm
@@ -1 +1 @@
-{"type":"MultiPoint","coordinates":[[1,1],[1,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
\ No newline at end of file
+{"type":"MultiPoint","coordinates":[[1,1],[1,2]]}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/secondary-heterogeneous-indexnl-params/secondary-heterogeneous-indexnl-params.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/secondary-heterogeneous-indexnl-params/secondary-heterogeneous-indexnl-params.3.adm
new file mode 100644
index 0000000..0290fae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/secondary-heterogeneous-indexnl-params/secondary-heterogeneous-indexnl-params.3.adm
@@ -0,0 +1 @@
+8800
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/secondary-heterogeneous-indexnl-params/secondary-heterogeneous-indexnl-params.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/secondary-heterogeneous-indexnl-params/secondary-heterogeneous-indexnl-params.4.adm
new file mode 100644
index 0000000..adffc32
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/secondary-heterogeneous-indexnl-params/secondary-heterogeneous-indexnl-params.4.adm
@@ -0,0 +1,2 @@
+1891
+8800
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-Leftouterjoin/heterogeneous-index-Leftouterjoin.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-Leftouterjoin/heterogeneous-index-Leftouterjoin.3.adm
new file mode 100644
index 0000000..5237c63
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-Leftouterjoin/heterogeneous-index-Leftouterjoin.3.adm
@@ -0,0 +1 @@
+{ "t1_id": "o_untyped:01" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-Leftouterjoin/heterogeneous-index-Leftouterjoin.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-Leftouterjoin/heterogeneous-index-Leftouterjoin.4.adm
new file mode 100644
index 0000000..5237c63
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-Leftouterjoin/heterogeneous-index-Leftouterjoin.4.adm
@@ -0,0 +1 @@
+{ "t1_id": "o_untyped:01" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-bulkLoad/heterogeneous-index-select.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-bulkLoad/heterogeneous-index-select.4.adm
new file mode 100644
index 0000000..cf8c3b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-bulkLoad/heterogeneous-index-select.4.adm
@@ -0,0 +1,3 @@
+{ "u": { "userId": 1, "email": "john.doe@example.com", "username": "johndoe", "isActive": true, "occupation": "teacher", "age": 23 } }
+{ "u": { "userId": 4, "email": "emily.davis@example.com", "username": "emilyd", "isActive": true, "age": 23.1 } }
+{ "u": { "userId": 8, "email": "robert.lee@example.com", "username": "robertl", "isActive": true, "age": 31 } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-select/heterogeneous-index-select.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-select/heterogeneous-index-select.4.adm
new file mode 100644
index 0000000..cf8c3b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-select/heterogeneous-index-select.4.adm
@@ -0,0 +1,3 @@
+{ "u": { "userId": 1, "email": "john.doe@example.com", "username": "johndoe", "isActive": true, "occupation": "teacher", "age": 23 } }
+{ "u": { "userId": 4, "email": "emily.davis@example.com", "username": "emilyd", "isActive": true, "age": 23.1 } }
+{ "u": { "userId": 8, "email": "robert.lee@example.com", "username": "robertl", "isActive": true, "age": 31 } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-select/heterogeneous-index-select.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-select/heterogeneous-index-select.5.adm
new file mode 100644
index 0000000..049b69e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-select/heterogeneous-index-select.5.adm
@@ -0,0 +1,3 @@
+{ "u": { "userId": 1, "email": "john.doe@example.com", "username": "johndoe", "isActive": true, "occupation": "teacher", "age": 23 } }
+{ "u": { "userId": 4, "email": "emily.davis@example.com", "username": "emilyd", "isActive": true, "age": 23.1 } }
+{ "u": { "userId": 8, "email": "robert.lee@example.com", "username": "robertl", "isActive": true, "age": 31 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-select/heterogeneous-index-select.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-select/heterogeneous-index-select.6.adm
new file mode 100644
index 0000000..5f0696b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-select/heterogeneous-index-select.6.adm
@@ -0,0 +1 @@
+{ "u": { "userId": 11, "email": "jack.moore@example.com", "username": "jk", "isActive": true, "age": "20", "occupation": "firefighter" } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-select/heterogeneous-index-select.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-select/heterogeneous-index-select.7.adm
new file mode 100644
index 0000000..5f0696b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/heterogeneous-index-select/heterogeneous-index-select.7.adm
@@ -0,0 +1 @@
+{ "u": { "userId": 11, "email": "jack.moore@example.com", "username": "jk", "isActive": true, "age": "20", "occupation": "firefighter" } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.03.adm
new file mode 100644
index 0000000..07e109a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.03.adm
@@ -0,0 +1 @@
+{ "ds1": { "id": 6, "name": "fsdxv😀", "age": 17 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.04.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.04.plan
new file mode 100644
index 0000000..a603091
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.04.plan
@@ -0,0 +1,34 @@
+distribute result [$$19] [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|
+    assign [$$19] <- [{"ds1": $$ds1}] project: [$$19] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (and(ge($$20, "fsdxv😀"), lt($$20, "fsdxv😁"))) project: [$$ds1] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        assign [$$20] <- [$$ds1.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- ASSIGN  |PARTITIONED|
+          project ([$$ds1]) [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, $$ds1] <- index-search("ds1", 0, "Default", "test", "ds1", false, false, 1, $$25, 1, $$25, true, true, true) [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, $$25) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      project ([$$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 [$$24, $$25] <- index-search("ds1_name", 0, "Default", "test", "ds1", false, false, 1, $$22, 1, $$23, 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 [$$22, $$23] <- ["fsdxv😀", "fsdxv😁"] [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/index-selection/like-expression/like-expression.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.05.adm
new file mode 100644
index 0000000..45d43d5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.05.adm
@@ -0,0 +1 @@
+{ "ds1": { "id": 4, "name": "asdgdh👩‍👩‍👧dfsd", "age": 23 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.06.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.06.plan
new file mode 100644
index 0000000..8b3870b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.06.plan
@@ -0,0 +1,34 @@
+distribute result [$$19] [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|
+    assign [$$19] <- [{"ds1": $$ds1}] project: [$$19] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (and(ge($$20, "asdgdh👩‍👩‍👧"), lt($$20, "asdgdh👩‍👩‍👨"))) project: [$$ds1] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        assign [$$20] <- [$$ds1.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- ASSIGN  |PARTITIONED|
+          project ([$$ds1]) [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, $$ds1] <- index-search("ds1", 0, "Default", "test", "ds1", false, false, 1, $$25, 1, $$25, true, true, true) [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, $$25) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      project ([$$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 [$$24, $$25] <- index-search("ds1_name", 0, "Default", "test", "ds1", false, false, 1, $$22, 1, $$23, 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 [$$22, $$23] <- ["asdgdh👩‍👩‍👧", "asdgdh👩‍👩‍👨"] [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/index-selection/like-expression/like-expression.07.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.07.adm
new file mode 100644
index 0000000..e4ac10b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.07.adm
@@ -0,0 +1 @@
+{ "ds1": { "id": 5, "name": "sxvciis", "age": 13 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.08.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.08.plan
new file mode 100644
index 0000000..9718efa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.08.plan
@@ -0,0 +1,32 @@
+distribute result [$$16] [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|
+    assign [$$16] <- [{"ds1": $$ds1}] project: [$$16] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (eq($$ds1.getField(1), "sxvciis")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$ds1]) [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, $$ds1] <- index-search("ds1", 0, "Default", "test", "ds1", false, false, 1, $$22, 1, $$22, true, true, true) [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, $$22) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    project ([$$22]) [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] <- index-search("ds1_name", 0, "Default", "test", "ds1", false, false, 1, $$19, 1, $$20, true, true, true) [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, $$20] <- ["sxvciis", "sxvciis"] [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/index-selection/like-expression/like-expression.09.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.09.adm
new file mode 100644
index 0000000..4ca49aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.09.adm
@@ -0,0 +1 @@
+{ "ds1": { "id": 1, "name": "John", "age": 21 } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.10.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.10.plan
new file mode 100644
index 0000000..b3c96d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.10.plan
@@ -0,0 +1,34 @@
+distribute result [$$19] [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|
+    assign [$$19] <- [{"ds1": $$ds1}] project: [$$19] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (and(ge($$20, "Joh"), lt($$20, "Joi"))) project: [$$ds1] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        assign [$$20] <- [$$ds1.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- ASSIGN  |PARTITIONED|
+          project ([$$ds1]) [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, $$ds1] <- index-search("ds1", 0, "Default", "test", "ds1", false, false, 1, $$25, 1, $$25, true, true, true) [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, $$25) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      project ([$$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 [$$24, $$25] <- index-search("ds1_name", 0, "Default", "test", "ds1", false, false, 1, $$22, 1, $$23, 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 [$$22, $$23] <- ["Joh", "Joi"] [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/index-selection/like-expression/like-expression.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.11.adm
new file mode 100644
index 0000000..202bfb1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.11.adm
@@ -0,0 +1 @@
+{ "ds1": { "id": 9, "name": "sxvfjshfjisciis", "age": 13 } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.12.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.12.plan
new file mode 100644
index 0000000..39ca4a6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.12.plan
@@ -0,0 +1,18 @@
+distribute result [$$16] [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|
+    assign [$$16] <- [{"ds1": $$ds1}] project: [$$16] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (like($$ds1.getField(1), "s%vfjs__jiscii%")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$ds1]) [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 []<-[$$17, $$ds1] <- test.ds1 [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/index-selection/like-expression/like-expression.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.13.adm
new file mode 100644
index 0000000..6bbea7c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.13.adm
@@ -0,0 +1 @@
+{ "ds1": { "id": 5, "name": "sxvciis", "age": 13 } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.14.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.14.plan
new file mode 100644
index 0000000..4483716
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.14.plan
@@ -0,0 +1,18 @@
+distribute result [$$16] [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|
+    assign [$$16] <- [{"ds1": $$ds1}] project: [$$16] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (like($$ds1.getField(1), "sxvc_is")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$ds1]) [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 []<-[$$17, $$ds1] <- test.ds1 [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/index-selection/like-expression/like-expression.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.15.adm
new file mode 100644
index 0000000..6bbea7c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.15.adm
@@ -0,0 +1 @@
+{ "ds1": { "id": 5, "name": "sxvciis", "age": 13 } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.16.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.16.plan
new file mode 100644
index 0000000..a318591
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.16.plan
@@ -0,0 +1,18 @@
+distribute result [$$16] [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|
+    assign [$$16] <- [{"ds1": $$ds1}] project: [$$16] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (like($$ds1.getField(1), "sxvc_i%")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$ds1]) [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 []<-[$$17, $$ds1] <- test.ds1 [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/index-selection/like-expression/like-expression.17.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.17.adm
new file mode 100644
index 0000000..202bfb1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.17.adm
@@ -0,0 +1 @@
+{ "ds1": { "id": 9, "name": "sxvfjshfjisciis", "age": 13 } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.18.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.18.plan
new file mode 100644
index 0000000..92964ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.18.plan
@@ -0,0 +1,18 @@
+distribute result [$$16] [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|
+    assign [$$16] <- [{"ds1": $$ds1}] project: [$$16] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (like($$ds1.getField(1), "sxvfjs%jisci%")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$ds1]) [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 []<-[$$17, $$ds1] <- test.ds1 [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/index-selection/like-expression/like-expression.19.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.19.adm
new file mode 100644
index 0000000..3d747c3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.19.adm
@@ -0,0 +1,2 @@
+{ "ds1": { "id": 1, "name": "John", "age": 21 } }
+{ "ds1": { "id": 8, "name": "Join", "age": 21 } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.20.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.20.plan
new file mode 100644
index 0000000..34953c3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.20.plan
@@ -0,0 +1,34 @@
+distribute result [$$19] [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|
+    assign [$$19] <- [{"ds1": $$ds1}] project: [$$19] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (and(ge($$20, "J"), lt($$20, "K"))) project: [$$ds1] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        assign [$$20] <- [$$ds1.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- ASSIGN  |PARTITIONED|
+          project ([$$ds1]) [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, $$ds1] <- index-search("ds1", 0, "Default", "test", "ds1", false, false, 1, $$25, 1, $$25, true, true, true) [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, $$25) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      project ([$$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 [$$24, $$25] <- index-search("ds1_name", 0, "Default", "test", "ds1", false, false, 1, $$22, 1, $$23, 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 [$$22, $$23] <- ["J", "K"] [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/index-selection/like-expression/like-expression.21.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.21.adm
new file mode 100644
index 0000000..be3fef3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.21.adm
@@ -0,0 +1,2 @@
+{ "ds1": { "id": 9, "name": "sxvfjshfjisciis", "age": 13 } }
+{ "ds1": { "id": 5, "name": "sxvciis", "age": 13 } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.22.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.22.plan
new file mode 100644
index 0000000..a4f11c1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.22.plan
@@ -0,0 +1,34 @@
+distribute result [$$19] [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|
+    assign [$$19] <- [{"ds1": $$ds1}] project: [$$19] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (and(ge($$20, "sx"), lt($$20, "sy"))) project: [$$ds1] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        assign [$$20] <- [$$ds1.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- ASSIGN  |PARTITIONED|
+          project ([$$ds1]) [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, $$ds1] <- index-search("ds1", 0, "Default", "test", "ds1", false, false, 1, $$25, 1, $$25, true, true, true) [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, $$25) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      project ([$$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 [$$24, $$25] <- index-search("ds1_name", 0, "Default", "test", "ds1", false, false, 1, $$22, 1, $$23, 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 [$$22, $$23] <- ["sx", "sy"] [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/index-selection/like-expression/like-expression.23.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.23.adm
new file mode 100644
index 0000000..77eef00
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.23.adm
@@ -0,0 +1 @@
+{ "t1": [ { "p": { "x": "fvxb%" } } ], "t2": [ { "p": { "x": "fv%%\\bn\\%" } } ], "t3": [  ], "t4": [ { "p": { "x": "fv%xbx" } } ], "t5": [ { "p": { "x": "a\\%bcd" } } ], "t6": [ { "p": { "x": "abc\n%" } } ], "t7": [ { "p": { "x": "abc\n%" } } ], "t8": [ { "p": { "x": "abc%cbz" } }, { "p": { "x": "abc%%x" } } ], "t9": [ { "p": { "x": "vhjdbvsabc_fjdhf" } } ], "t10": [ { "p": { "x": "abc_fjdhf" } } ], "t11": [ { "p": { "x": "abc%x" } } ], "t12": [ { "p": { "x": "abc_" } } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.24.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.24.adm
new file mode 100644
index 0000000..2ba2846
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.24.adm
@@ -0,0 +1 @@
+{ "ds1": { "id": 10, "name": "h\\axx%", "age": 123 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.25.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.25.plan
new file mode 100644
index 0000000..a06a7c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/like-expression/like-expression.25.plan
@@ -0,0 +1,32 @@
+distribute result [$$16] [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|
+    assign [$$16] <- [{"ds1": $$ds1}] project: [$$16] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (eq($$ds1.getField(1), "h\axx%")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$ds1]) [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, $$ds1] <- index-search("ds1", 0, "Default", "test", "ds1", false, false, 1, $$22, 1, $$22, true, true, true) [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, $$22) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    project ([$$22]) [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] <- index-search("ds1_name", 0, "Default", "test", "ds1", false, false, 1, $$19, 1, $$20, true, true, true) [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, $$20] <- ["h\axx%", "h\axx%"] [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/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 453c83d..ab03c07 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
@@ -2,63 +2,57 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$51]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$51] <- [{"n_nationkey": $$58, "s_nationkey": $$56, "c_nationkey": $$55}] [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 [$$58(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
-          order (ASC, $$58) (ASC, $$56) (ASC, $$55) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$58(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              join (eq($$55, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HYBRID_HASH_JOIN [$$58][$$55]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  join (eq($$56, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HYBRID_HASH_JOIN [$$58][$$56]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-                      project ([$$58]) [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 []<-[$$58, $$n] <- tpch.Nation [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 [$$56]  |PARTITIONED|
-                      project ([$$56]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        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 []<-[$$59, $$s] <- tpch.Supplier [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 [$$55]  |PARTITIONED|
-                  project ([$$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$55] <- [$$c.getField(3)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$51] <- [{"n_nationkey": $$58, "s_nationkey": $$56, "c_nationkey": $$55}] project: [$$51] [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 [$$58(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
+        order (ASC, $$58) (ASC, $$56) (ASC, $$55) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$58(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            join (eq($$55, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HYBRID_HASH_JOIN [$$58][$$55]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                join (eq($$56, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HYBRID_HASH_JOIN [$$58][$$56]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
+                    project ([$$58]) [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 []<-[$$58, $$n] <- tpch.Nation [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 [$$56]  |PARTITIONED|
+                    assign [$$56] <- [$$s.getField(3)] project: [$$56] [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]
+                      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 []<-[$$60, $$c] <- tpch.Customer [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          data-scan []<-[$$59, $$s] <- tpch.Supplier [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 [$$55]  |PARTITIONED|
+                assign [$$55] <- [$$c.getField(3)] project: [$$55] [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|
+                      data-scan []<-[$$60, $$c] <- tpch.Customer [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/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 025e283..d107789 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
@@ -2,49 +2,45 @@
 -- 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]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$36] <- [{"o_orderkey": $$43, "l_orderkey": $$44, "l_suppkey": $$42}] [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 [$$43(ASC), $$44(ASC), $$42(ASC) ]  |PARTITIONED|
-          order (ASC, $$43) (ASC, $$44) (ASC, $$42) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$43(ASC), $$44(ASC), $$42(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$43, $$44, $$42]) [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|
-                  join (and(eq($$43, $$44), eq($$49, $$42))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HYBRID_HASH_JOIN [$$43, $$49][$$44, $$42]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HASH_PARTITION_EXCHANGE [$$43, $$49]  |PARTITIONED|
-                      assign [$$49] <- [$$43] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
-                        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 []<-[$$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 [$$44, $$42]  |PARTITIONED|
-                      project ([$$44, $$42]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$36] <- [{"o_orderkey": $$43, "l_orderkey": $$44, "l_suppkey": $$42}] project: [$$36] [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 [$$43(ASC), $$44(ASC), $$42(ASC) ]  |PARTITIONED|
+        order (ASC, $$43) (ASC, $$44) (ASC, $$42) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$43(ASC), $$44(ASC), $$42(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            project ([$$43, $$44, $$42]) [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|
+                join (and(eq($$43, $$44), eq($$49, $$42))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HYBRID_HASH_JOIN [$$43, $$49][$$44, $$42]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HASH_PARTITION_EXCHANGE [$$43, $$49]  |PARTITIONED|
+                    assign [$$49] <- [$$43] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ASSIGN  |PARTITIONED|
+                      project ([$$43]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$42] <- [$$l.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ASSIGN  |PARTITIONED|
-                          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 []<-[$$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|
-                              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|
-                                  empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- EMPTY_TUPLE_SOURCE  |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 [$$44, $$42]  |PARTITIONED|
+                    assign [$$42] <- [$$l.getField(2)] project: [$$44, $$42] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ASSIGN  |PARTITIONED|
+                      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 []<-[$$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|
+                              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.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 61de808..3122528 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
@@ -2,49 +2,45 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$38]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      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 [$$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 ([$$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($$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]
-                    -- HASH_PARTITION_EXCHANGE [$$43, $$56]  |PARTITIONED|
-                      assign [$$56] <- [$$43] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
-                        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 []<-[$$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 [$$44, $$47]  |PARTITIONED|
-                      project ([$$44, $$47]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$38] <- [{"o_orderkey": $$43, "l_orderkey": $$44, "l_suppkey": $$47}] project: [$$38] [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 [$$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 ([$$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($$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]
+                  -- HASH_PARTITION_EXCHANGE [$$43, $$56]  |PARTITIONED|
+                    assign [$$56] <- [$$43] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ASSIGN  |PARTITIONED|
+                      project ([$$43]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$47] <- [$$l.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ASSIGN  |PARTITIONED|
-                          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 []<-[$$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|
-                              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|
-                                  empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- EMPTY_TUPLE_SOURCE  |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 [$$44, $$47]  |PARTITIONED|
+                    assign [$$47] <- [$$l.getField(2)] project: [$$44, $$47] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ASSIGN  |PARTITIONED|
+                      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 []<-[$$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|
+                              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.10.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.10.plan
index 453c83d..ab03c07 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.10.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.10.plan
@@ -2,63 +2,57 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$51]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$51] <- [{"n_nationkey": $$58, "s_nationkey": $$56, "c_nationkey": $$55}] [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 [$$58(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
-          order (ASC, $$58) (ASC, $$56) (ASC, $$55) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$58(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              join (eq($$55, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HYBRID_HASH_JOIN [$$58][$$55]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  join (eq($$56, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HYBRID_HASH_JOIN [$$58][$$56]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-                      project ([$$58]) [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 []<-[$$58, $$n] <- tpch.Nation [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 [$$56]  |PARTITIONED|
-                      project ([$$56]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        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 []<-[$$59, $$s] <- tpch.Supplier [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 [$$55]  |PARTITIONED|
-                  project ([$$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$55] <- [$$c.getField(3)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$51] <- [{"n_nationkey": $$58, "s_nationkey": $$56, "c_nationkey": $$55}] project: [$$51] [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 [$$58(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
+        order (ASC, $$58) (ASC, $$56) (ASC, $$55) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$58(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            join (eq($$55, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HYBRID_HASH_JOIN [$$58][$$55]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                join (eq($$56, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HYBRID_HASH_JOIN [$$58][$$56]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
+                    project ([$$58]) [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 []<-[$$58, $$n] <- tpch.Nation [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 [$$56]  |PARTITIONED|
+                    assign [$$56] <- [$$s.getField(3)] project: [$$56] [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]
+                      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 []<-[$$60, $$c] <- tpch.Customer [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          data-scan []<-[$$59, $$s] <- tpch.Supplier [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 [$$55]  |PARTITIONED|
+                assign [$$55] <- [$$c.getField(3)] project: [$$55] [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|
+                      data-scan []<-[$$60, $$c] <- tpch.Customer [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/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.12.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.12.plan
index 453c83d..ab03c07 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.12.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.12.plan
@@ -2,63 +2,57 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$51]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$51] <- [{"n_nationkey": $$58, "s_nationkey": $$56, "c_nationkey": $$55}] [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 [$$58(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
-          order (ASC, $$58) (ASC, $$56) (ASC, $$55) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$58(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              join (eq($$55, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HYBRID_HASH_JOIN [$$58][$$55]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  join (eq($$56, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HYBRID_HASH_JOIN [$$58][$$56]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-                      project ([$$58]) [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 []<-[$$58, $$n] <- tpch.Nation [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 [$$56]  |PARTITIONED|
-                      project ([$$56]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        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 []<-[$$59, $$s] <- tpch.Supplier [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 [$$55]  |PARTITIONED|
-                  project ([$$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$55] <- [$$c.getField(3)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$51] <- [{"n_nationkey": $$58, "s_nationkey": $$56, "c_nationkey": $$55}] project: [$$51] [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 [$$58(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
+        order (ASC, $$58) (ASC, $$56) (ASC, $$55) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$58(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            join (eq($$55, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HYBRID_HASH_JOIN [$$58][$$55]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                join (eq($$56, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HYBRID_HASH_JOIN [$$58][$$56]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
+                    project ([$$58]) [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 []<-[$$58, $$n] <- tpch.Nation [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 [$$56]  |PARTITIONED|
+                    assign [$$56] <- [$$s.getField(3)] project: [$$56] [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]
+                      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 []<-[$$60, $$c] <- tpch.Customer [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          data-scan []<-[$$59, $$s] <- tpch.Supplier [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 [$$55]  |PARTITIONED|
+                assign [$$55] <- [$$c.getField(3)] project: [$$55] [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|
+                      data-scan []<-[$$60, $$c] <- tpch.Customer [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/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 b1042e0..b00473e 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
@@ -2,65 +2,59 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$51]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$51] <- [{"n_nationkey": $$59, "s_nationkey": $$56, "c_nationkey": $$55}] [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 [$$59(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
-          order (ASC, $$59) (ASC, $$56) (ASC, $$55) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$59(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$59, $$56, $$55]) [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|
-                  join (and(eq($$55, $$59), eq($$56, $$66))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HYBRID_HASH_JOIN [$$59, $$56][$$55, $$66]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HASH_PARTITION_EXCHANGE [$$59, $$56]  |PARTITIONED|
-                      project ([$$56, $$59]) [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 [$$59, $$n] <- index-search("Nation", 0, "Default", "tpch", "Nation", true, true, 1, $$56, 1, $$56, true, true, true) [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, $$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 [$$56]  |PARTITIONED|
-                                  project ([$$56]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$51] <- [{"n_nationkey": $$59, "s_nationkey": $$56, "c_nationkey": $$55}] project: [$$51] [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 [$$59(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
+        order (ASC, $$59) (ASC, $$56) (ASC, $$55) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$59(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            project ([$$59, $$56, $$55]) [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|
+                join (and(eq($$55, $$59), eq($$56, $$66))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HYBRID_HASH_JOIN [$$59, $$56][$$55, $$66]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HASH_PARTITION_EXCHANGE [$$59, $$56]  |PARTITIONED|
+                    project ([$$56, $$59]) [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 [$$59, $$n] <- index-search("Nation", 0, "Default", "tpch", "Nation", true, true, 1, $$56, 1, $$56, true, true, true) [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, $$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 [$$56]  |PARTITIONED|
+                                assign [$$56] <- [$$s.getField(3)] project: [$$56] [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|
-                                    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 []<-[$$58, $$s] <- tpch.Supplier [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|
-                                          data-scan []<-[$$58, $$s] <- tpch.Supplier [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 [$$55, $$66]  |PARTITIONED|
-                      assign [$$66] <- [$$55] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          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 [$$55, $$66]  |PARTITIONED|
+                    assign [$$66] <- [$$55] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ASSIGN  |PARTITIONED|
+                      assign [$$55] <- [$$c.getField(3)] project: [$$55] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- ASSIGN  |PARTITIONED|
-                        project ([$$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        project ([$$c]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$55] <- [$$c.getField(3)] [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|
+                            data-scan []<-[$$60, $$c] <- tpch.Customer [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|
-                                data-scan []<-[$$60, $$c] <- tpch.Customer [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|
+                                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.16.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.16.plan
index b1042e0..b00473e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.16.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.16.plan
@@ -2,65 +2,59 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$51]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$51] <- [{"n_nationkey": $$59, "s_nationkey": $$56, "c_nationkey": $$55}] [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 [$$59(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
-          order (ASC, $$59) (ASC, $$56) (ASC, $$55) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$59(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$59, $$56, $$55]) [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|
-                  join (and(eq($$55, $$59), eq($$56, $$66))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HYBRID_HASH_JOIN [$$59, $$56][$$55, $$66]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HASH_PARTITION_EXCHANGE [$$59, $$56]  |PARTITIONED|
-                      project ([$$56, $$59]) [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 [$$59, $$n] <- index-search("Nation", 0, "Default", "tpch", "Nation", true, true, 1, $$56, 1, $$56, true, true, true) [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, $$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 [$$56]  |PARTITIONED|
-                                  project ([$$56]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$51] <- [{"n_nationkey": $$59, "s_nationkey": $$56, "c_nationkey": $$55}] project: [$$51] [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 [$$59(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
+        order (ASC, $$59) (ASC, $$56) (ASC, $$55) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$59(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            project ([$$59, $$56, $$55]) [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|
+                join (and(eq($$55, $$59), eq($$56, $$66))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HYBRID_HASH_JOIN [$$59, $$56][$$55, $$66]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HASH_PARTITION_EXCHANGE [$$59, $$56]  |PARTITIONED|
+                    project ([$$56, $$59]) [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 [$$59, $$n] <- index-search("Nation", 0, "Default", "tpch", "Nation", true, true, 1, $$56, 1, $$56, true, true, true) [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, $$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 [$$56]  |PARTITIONED|
+                                assign [$$56] <- [$$s.getField(3)] project: [$$56] [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|
-                                    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 []<-[$$58, $$s] <- tpch.Supplier [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|
-                                          data-scan []<-[$$58, $$s] <- tpch.Supplier [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 [$$55, $$66]  |PARTITIONED|
-                      assign [$$66] <- [$$55] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          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 [$$55, $$66]  |PARTITIONED|
+                    assign [$$66] <- [$$55] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ASSIGN  |PARTITIONED|
+                      assign [$$55] <- [$$c.getField(3)] project: [$$55] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- ASSIGN  |PARTITIONED|
-                        project ([$$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        project ([$$c]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$55] <- [$$c.getField(3)] [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|
+                            data-scan []<-[$$60, $$c] <- tpch.Customer [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|
-                                data-scan []<-[$$60, $$c] <- tpch.Customer [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|
+                                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/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.007.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.007.plan
index 0d91c33..2aa1aec 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.007.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.007.plan
@@ -2,61 +2,53 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$52] <- [{"t1_id": $$53, "t2_id": $$54}] [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 [$$53(ASC), $$54(ASC) ]  |PARTITIONED|
-          order (ASC, $$53) (ASC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$53, $$54]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select ($$62) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  window-aggregate [$$62] <- [win-mark-first-missing-impl($$54)] partition [$$53] order (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- WINDOW_STREAM  |PARTITIONED|
+    assign [$$52] <- [{"t1_id": $$53, "t2_id": $$54}] project: [$$52] [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 [$$53(ASC), $$54(ASC) ]  |PARTITIONED|
+        order (ASC, $$53) (ASC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select ($$62) project: [$$53, $$54] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              window-aggregate [$$62] <- [win-mark-first-missing-impl($$54)] partition [$$53] order (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- WINDOW_STREAM  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$53) (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STABLE_SORT [$$53(ASC), $$54(DESC)]  |PARTITIONED|
                     exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      order (ASC, $$53) (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STABLE_SORT [$$53(ASC), $$54(DESC)]  |PARTITIONED|
-                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
-                          project ([$$53, $$54]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            select (eq($$55, $$t2.getField("c_int32"))) retain-untrue ($$54 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_SELECT  |PARTITIONED|
-                              project ([$$53, $$55, $$54, $$t2]) [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-unnest-map [$$54, $$t2] <- index-search("ds_inner_untyped", 0, "Default", "test", "ds_inner_untyped", true, false, 1, $$61, 1, $$61, true, true, true) [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, $$61) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
-                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          project ([$$53, $$55, $$61]) [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-unnest-map [$$60, $$61] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_untyped", true, true, 1, $$55, 1, $$55, true, true, true) [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]
-                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                  project ([$$53, $$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    assign [$$55] <- [int32($$t1.getField("c_int32"))] [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]
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped [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|
+                    -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                      select (eq($$55, $$t2.getField("c_int32"))) retain-untrue ($$54 <- missing) project: [$$53, $$54] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- STREAM_SELECT  |PARTITIONED|
+                        project ([$$53, $$55, $$54, $$t2]) [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-unnest-map [$$54, $$t2] <- index-search("ds_inner_untyped", 0, "Default", "test", "ds_inner_untyped", true, false, 1, $$61, 1, $$61, true, true, true) [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, $$61) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
+                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    project ([$$53, $$55, $$61]) [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-unnest-map [$$60, $$61] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_untyped", true, true, 1, $$55, 1, $$55, true, true, true) [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]
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            assign [$$55] <- [int32($$t1.getField("c_int32"))] project: [$$53, $$55] [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]
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped [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/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.008.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.008.plan
index 8726b8d..5d0a159 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.008.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.008.plan
@@ -2,73 +2,37 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$52] <- [{"t1_id": $$73, "t2_id": $$54}] [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 [$$73(ASC), $$54(ASC) ]  |PARTITIONED|
-          order (ASC, $$73) (ASC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$73(ASC), $$54(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$73, $$54]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select ($$74) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  window-aggregate [$$74] <- [win-mark-first-missing-impl($$54)] partition [$$73] order (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- WINDOW_STREAM  |PARTITIONED|
+    assign [$$52] <- [{"t1_id": $$73, "t2_id": $$54}] project: [$$52] [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 [$$73(ASC), $$54(ASC) ]  |PARTITIONED|
+        order (ASC, $$73) (ASC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$73(ASC), $$54(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select ($$74) project: [$$73, $$54] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              window-aggregate [$$74] <- [win-mark-first-missing-impl($$54)] partition [$$73] order (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- WINDOW_STREAM  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$73) (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STABLE_SORT [$$73(ASC), $$54(DESC)]  |PARTITIONED|
                     exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      order (ASC, $$73) (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STABLE_SORT [$$73(ASC), $$54(DESC)]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$73]  |PARTITIONED|
+                      union ($$70, $$61, $$54) ($$53, $$53, $$73) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- UNION_ALL  |PARTITIONED|
                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HASH_PARTITION_EXCHANGE [$$73]  |PARTITIONED|
-                          union ($$70, $$61, $$54) ($$53, $$53, $$73) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- UNION_ALL  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          select (eq($$55, $$71.getField(1))) retain-untrue ($$70 <- missing) project: [$$70, $$53] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- STREAM_SELECT  |PARTITIONED|
                             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              project ([$$70, $$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                select (eq($$55, $$71.getField(1))) retain-untrue ($$70 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    left-outer-unnest-map [$$70, $$71] <- index-search("ds_inner_typed", 0, "Default", "test", "ds_inner_typed", true, false, 1, $$61, 1, $$61, true, true, true) [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|
-                                        project ([$$53, $$55, $$61]) [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|
-                                            split ($$62) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- SPLIT  |PARTITIONED|
-                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                left-outer-unnest-map [$$60, $$61, $$62] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_typed", true, true, 1, $$55, 1, $$55, true, true, true) [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]
-                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                    project ([$$53, $$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      assign [$$55] <- [int32($$t1.getField("c_int32"))] [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]
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped [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  |PARTITIONED|
-                              project ([$$61, $$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                select (eq($$55, $$60)) retain-untrue ($$61 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  project ([$$53, $$55, $$60, $$61]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              left-outer-unnest-map [$$70, $$71] <- index-search("ds_inner_typed", 0, "Default", "test", "ds_inner_typed", true, false, 1, $$61, 1, $$61, true, true, true) [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|
+                                  project ([$$53, $$55, $$61]) [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|
@@ -80,15 +44,39 @@
                                           -- BTREE_SEARCH  |PARTITIONED|
                                             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                             -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                              project ([$$53, $$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                assign [$$55] <- [int32($$t1.getField("c_int32"))] [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]
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped [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|
+                                              assign [$$55] <- [int32($$t1.getField("c_int32"))] project: [$$53, $$55] [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]
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped [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  |PARTITIONED|
+                          select (eq($$55, $$60)) retain-untrue ($$61 <- missing) project: [$$61, $$53] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- STREAM_SELECT  |PARTITIONED|
+                            project ([$$53, $$55, $$60, $$61]) [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|
+                                split ($$62) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- SPLIT  |PARTITIONED|
+                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    left-outer-unnest-map [$$60, $$61, $$62] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_typed", true, true, 1, $$55, 1, $$55, true, true, true) [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]
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        assign [$$55] <- [int32($$t1.getField("c_int32"))] project: [$$53, $$55] [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]
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped [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/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.009.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.009.plan
index 0465493..0bfde17 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.009.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.009.plan
@@ -2,61 +2,53 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$52] <- [{"t1_id": $$53, "t2_id": $$54}] [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 [$$53(ASC), $$54(ASC) ]  |PARTITIONED|
-          order (ASC, $$53) (ASC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$53, $$54]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select ($$62) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  window-aggregate [$$62] <- [win-mark-first-missing-impl($$54)] partition [$$53] order (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- WINDOW_STREAM  |PARTITIONED|
+    assign [$$52] <- [{"t1_id": $$53, "t2_id": $$54}] project: [$$52] [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 [$$53(ASC), $$54(ASC) ]  |PARTITIONED|
+        order (ASC, $$53) (ASC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select ($$62) project: [$$53, $$54] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              window-aggregate [$$62] <- [win-mark-first-missing-impl($$54)] partition [$$53] order (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- WINDOW_STREAM  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$53) (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STABLE_SORT [$$53(ASC), $$54(DESC)]  |PARTITIONED|
                     exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      order (ASC, $$53) (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STABLE_SORT [$$53(ASC), $$54(DESC)]  |PARTITIONED|
-                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
-                          project ([$$53, $$54]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            select (eq($$55, $$t2.getField("c_int32"))) retain-untrue ($$54 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_SELECT  |PARTITIONED|
-                              project ([$$53, $$55, $$54, $$t2]) [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-unnest-map [$$54, $$t2] <- index-search("ds_inner_untyped", 0, "Default", "test", "ds_inner_untyped", true, false, 1, $$61, 1, $$61, true, true, true) [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, $$61) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
-                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          project ([$$53, $$55, $$61]) [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-unnest-map [$$60, $$61] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_untyped", true, true, 1, $$55, 1, $$55, true, true, true) [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]
-                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                  project ([$$53, $$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    assign [$$55] <- [int32($$t1.getField(1))] [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]
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        data-scan []<-[$$53, $$t1] <- test.ds_outer_typed [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|
+                    -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                      select (eq($$55, $$t2.getField("c_int32"))) retain-untrue ($$54 <- missing) project: [$$53, $$54] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- STREAM_SELECT  |PARTITIONED|
+                        project ([$$53, $$55, $$54, $$t2]) [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-unnest-map [$$54, $$t2] <- index-search("ds_inner_untyped", 0, "Default", "test", "ds_inner_untyped", true, false, 1, $$61, 1, $$61, true, true, true) [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, $$61) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
+                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    project ([$$53, $$55, $$61]) [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-unnest-map [$$60, $$61] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_untyped", true, true, 1, $$55, 1, $$55, true, true, true) [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]
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            assign [$$55] <- [int32($$t1.getField(1))] project: [$$53, $$55] [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]
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                data-scan []<-[$$53, $$t1] <- test.ds_outer_typed [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/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.010.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.010.plan
index 2f3540a..b936dff 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.010.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.010.plan
@@ -2,73 +2,37 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$52] <- [{"t1_id": $$73, "t2_id": $$54}] [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 [$$73(ASC), $$54(ASC) ]  |PARTITIONED|
-          order (ASC, $$73) (ASC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$73(ASC), $$54(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$73, $$54]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select ($$74) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  window-aggregate [$$74] <- [win-mark-first-missing-impl($$54)] partition [$$73] order (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- WINDOW_STREAM  |PARTITIONED|
+    assign [$$52] <- [{"t1_id": $$73, "t2_id": $$54}] project: [$$52] [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 [$$73(ASC), $$54(ASC) ]  |PARTITIONED|
+        order (ASC, $$73) (ASC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$73(ASC), $$54(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select ($$74) project: [$$73, $$54] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              window-aggregate [$$74] <- [win-mark-first-missing-impl($$54)] partition [$$73] order (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- WINDOW_STREAM  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$73) (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STABLE_SORT [$$73(ASC), $$54(DESC)]  |PARTITIONED|
                     exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      order (ASC, $$73) (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STABLE_SORT [$$73(ASC), $$54(DESC)]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$73]  |PARTITIONED|
+                      union ($$70, $$61, $$54) ($$53, $$53, $$73) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- UNION_ALL  |PARTITIONED|
                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HASH_PARTITION_EXCHANGE [$$73]  |PARTITIONED|
-                          union ($$70, $$61, $$54) ($$53, $$53, $$73) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- UNION_ALL  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          select (eq($$55, $$71.getField(1))) retain-untrue ($$70 <- missing) project: [$$70, $$53] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- STREAM_SELECT  |PARTITIONED|
                             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              project ([$$70, $$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                select (eq($$55, $$71.getField(1))) retain-untrue ($$70 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    left-outer-unnest-map [$$70, $$71] <- index-search("ds_inner_typed", 0, "Default", "test", "ds_inner_typed", true, false, 1, $$61, 1, $$61, true, true, true) [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|
-                                        project ([$$53, $$55, $$61]) [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|
-                                            split ($$62) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- SPLIT  |PARTITIONED|
-                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                left-outer-unnest-map [$$60, $$61, $$62] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_typed", true, true, 1, $$55, 1, $$55, true, true, true) [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]
-                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                    project ([$$53, $$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      assign [$$55] <- [int32($$t1.getField(1))] [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]
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          data-scan []<-[$$53, $$t1] <- test.ds_outer_typed [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  |PARTITIONED|
-                              project ([$$61, $$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                select (eq($$55, $$60)) retain-untrue ($$61 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  project ([$$53, $$55, $$60, $$61]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              left-outer-unnest-map [$$70, $$71] <- index-search("ds_inner_typed", 0, "Default", "test", "ds_inner_typed", true, false, 1, $$61, 1, $$61, true, true, true) [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|
+                                  project ([$$53, $$55, $$61]) [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|
@@ -80,15 +44,39 @@
                                           -- BTREE_SEARCH  |PARTITIONED|
                                             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                             -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                              project ([$$53, $$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                assign [$$55] <- [int32($$t1.getField(1))] [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]
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    data-scan []<-[$$53, $$t1] <- test.ds_outer_typed [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|
+                                              assign [$$55] <- [int32($$t1.getField(1))] project: [$$53, $$55] [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]
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  data-scan []<-[$$53, $$t1] <- test.ds_outer_typed [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  |PARTITIONED|
+                          select (eq($$55, $$60)) retain-untrue ($$61 <- missing) project: [$$61, $$53] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- STREAM_SELECT  |PARTITIONED|
+                            project ([$$53, $$55, $$60, $$61]) [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|
+                                split ($$62) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- SPLIT  |PARTITIONED|
+                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    left-outer-unnest-map [$$60, $$61, $$62] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_typed", true, true, 1, $$55, 1, $$55, true, true, true) [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]
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        assign [$$55] <- [int32($$t1.getField(1))] project: [$$53, $$55] [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]
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            data-scan []<-[$$53, $$t1] <- test.ds_outer_typed [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/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 223e409..e520ad8 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
@@ -4,19 +4,17 @@
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit offset 98 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$16]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$16] <- [{"id": $$18, "dblpid": $$paper.getField(1)}] [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 [$$18(ASC) ]  |PARTITIONED|
-            order (ASC, $$18) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                data-scan []<-[$$18, $$paper] <- test.DBLP1 [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|
+      assign [$$16] <- [{"id": $$18, "dblpid": $$paper.getField(1)}] project: [$$16] [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 [$$18(ASC) ]  |PARTITIONED|
+          order (ASC, $$18) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$18, $$paper] <- test.DBLP1 [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-external-scan-select/push-limit-to-external-scan-select.2.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan-select/push-limit-to-external-scan-select.2.plan
index 11d4b1a..82aa37e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan-select/push-limit-to-external-scan-select.2.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan-select/push-limit-to-external-scan-select.2.plan
@@ -8,15 +8,13 @@
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
         limit 5 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
         -- STREAM_LIMIT  |PARTITIONED|
-          project ([$$17]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STREAM_PROJECT  |PARTITIONED|
-            assign [$$17] <- [$$t.getField(0)] [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]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                data-scan []<-[$$t] <- test.ds1 condition (gt($$t.getField(0), 2)) limit 5 [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|
+          assign [$$17] <- [$$t.getField(0)] project: [$$17] [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]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$t] <- test.ds1 condition (gt($$t.getField(0), 2)) limit 5 [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-external-scan/push-limit-to-external-scan.2.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan/push-limit-to-external-scan.2.plan
index 55b2c18..4428df0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan/push-limit-to-external-scan.2.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan/push-limit-to-external-scan.2.plan
@@ -6,17 +6,15 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        project ([$$14]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          assign [$$14] <- [$$t.getField(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|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                data-scan []<-[$$t] <- test.ds1 limit 5 [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|
+        assign [$$14] <- [$$t.getField(0)] project: [$$14] [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|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$t] <- test.ds1 limit 5 [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-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 3420b11..d080390 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
@@ -6,35 +6,33 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       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 [$$22, $$23, $$c] <- index-search("LineItem", 0, "Default", "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, $$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 ([$$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 [$$26, $$27, $$28] <- index-search("idx_LineItem_suppkey", 0, "Default", "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 [$$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|
+        assign [$$20] <- [{"shipdate": substring($$c.getField(10), 0, 4), "suppkey": gt($$21, 0)}] project: [$$20] [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 [$$22, $$23, $$c] <- index-search("LineItem", 0, "Default", "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, $$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 ([$$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 [$$26, $$27, $$28] <- index-search("idx_LineItem_suppkey", 0, "Default", "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 [$$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-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 072f7fe..3b3dec6 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
@@ -6,59 +6,51 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        project ([$$37]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          assign [$$37] <- [{"dblpid": $$38}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- ASSIGN  |PARTITIONED|
-            limit 2 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_LIMIT  |PARTITIONED|
-              project ([$$38]) [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|
-                  join (eq($$38, $$41)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HYBRID_HASH_JOIN [$$38][$$41]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
-                      project ([$$38]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        assign [$$37] <- [{"dblpid": $$38}] project: [$$37] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- ASSIGN  |PARTITIONED|
+          limit 2 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STREAM_LIMIT  |PARTITIONED|
+            project ([$$38]) [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|
+                join (eq($$38, $$41)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HYBRID_HASH_JOIN [$$38][$$41]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                    assign [$$38] <- [$$d.getField(1)] project: [$$38] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ASSIGN  |PARTITIONED|
+                      project ([$$d]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$38] <- [$$d.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ASSIGN  |PARTITIONED|
-                          project ([$$d]) [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 []<-[$$39, $$d] <- test.DBLP1 [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|
-                              data-scan []<-[$$39, $$d] <- test.DBLP1 [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 [$$41]  |PARTITIONED|
-                      project ([$$41]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_PROJECT  |UNPARTITIONED|
-                        assign [$$41] <- [get-item($$30, 0).getField(0).getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ASSIGN  |UNPARTITIONED|
-                          aggregate [$$30] <- [listify($$29)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |UNPARTITIONED|
-                            limit 1 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_LIMIT  |UNPARTITIONED|
-                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                project ([$$29]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              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 [$$41]  |PARTITIONED|
+                    assign [$$41] <- [get-item($$30, 0).getField(0).getField(1)] project: [$$41] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ASSIGN  |UNPARTITIONED|
+                      aggregate [$$30] <- [listify($$29)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |UNPARTITIONED|
+                        limit 1 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- STREAM_LIMIT  |UNPARTITIONED|
+                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                            assign [$$29] <- [{"d": $$d}] project: [$$29] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- ASSIGN  |PARTITIONED|
+                              limit 1 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- STREAM_LIMIT  |PARTITIONED|
+                                project ([$$d]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                 -- STREAM_PROJECT  |PARTITIONED|
-                                  assign [$$29] <- [{"d": $$d}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ASSIGN  |PARTITIONED|
-                                    limit 1 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- STREAM_LIMIT  |PARTITIONED|
-                                      project ([$$d]) [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 []<-[$$40, $$d] <- test.DBLP1 condition (ends-with($$d.getField(1), "Blakeley95")) limit 1 [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  |PARTITIONED|
+                                    data-scan []<-[$$40, $$d] <- test.DBLP1 condition (ends-with($$d.getField(1), "Blakeley95")) limit 1 [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.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 355ecbb..2eeb3ff 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
@@ -6,23 +6,19 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        project ([$$19]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          assign [$$19] <- [{"$1": substring($$20, 0, 21)}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- ASSIGN  |PARTITIONED|
-            limit 1 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_LIMIT  |PARTITIONED|
-              project ([$$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        assign [$$19] <- [{"$1": substring($$20, 0, 21)}] project: [$$19] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- ASSIGN  |PARTITIONED|
+          limit 1 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STREAM_LIMIT  |PARTITIONED|
+            assign [$$20] <- [$$DBLP1.getField(1)] project: [$$20] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ASSIGN  |PARTITIONED|
+              project ([$$DBLP1]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
               -- STREAM_PROJECT  |PARTITIONED|
-                assign [$$20] <- [$$DBLP1.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ASSIGN  |PARTITIONED|
-                  project ([$$DBLP1]) [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 []<-[$$21, $$DBLP1] <- test.DBLP1 condition (gt($$DBLP1.getField(1), "series")) limit 1 [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|
-                      data-scan []<-[$$21, $$DBLP1] <- test.DBLP1 condition (gt($$DBLP1.getField(1), "series")) limit 1 [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|
+                      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.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 0d2acd4..1090f18 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
@@ -8,21 +8,17 @@
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
         limit 2 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
         -- STREAM_LIMIT  |PARTITIONED|
-          project ([$$22]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STREAM_PROJECT  |PARTITIONED|
-            assign [$$22] <- [$$26.getField("lang")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          assign [$$22] <- [$$26.getField("lang")] project: [$$22] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ASSIGN  |PARTITIONED|
+            assign [$$26] <- [$$t.getField("user")] project: [$$26] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
             -- ASSIGN  |PARTITIONED|
-              project ([$$26]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              project ([$$t]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
               -- STREAM_PROJECT  |PARTITIONED|
-                assign [$$26] <- [$$t.getField("user")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ASSIGN  |PARTITIONED|
-                  project ([$$t]) [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 []<-[$$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: 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|
-                      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: 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|
+                      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 6496613..0c144ee 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
@@ -4,40 +4,38 @@
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit 5 offset 5 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$80]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$80] <- [get-item($$78, 0)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- ASSIGN  |PARTITIONED|
-          project ([$$78]) [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 [$$82(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]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  order (topK: 10) (ASC, $$82) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STABLE_SORT [topK: 10] [$$82(ASC)]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      project ([$$78, $$82]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.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: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- SUBPLAN  |PARTITIONED|
-                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            data-scan []<-[$$82, $$paper] <- test.DBLP1 [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|
+      assign [$$80] <- [get-item($$78, 0)] project: [$$80] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- ASSIGN  |PARTITIONED|
+        project ([$$78]) [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 [$$82(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]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                order (topK: 10) (ASC, $$82) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STABLE_SORT [topK: 10] [$$82(ASC)]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    project ([$$78, $$82]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.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: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- SUBPLAN  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$82, $$paper] <- test.DBLP1 [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/list/enforcing_item_type/enforcing_item_type.1.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/enforcing_item_type/enforcing_item_type.1.plan
index b0a9850..bb24314 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/list/enforcing_item_type/enforcing_item_type.1.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/enforcing_item_type/enforcing_item_type.1.plan
@@ -2,11 +2,9 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$21]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$21] <- [{"id": get-item($$t, 0), "v": get-item($$t, 1)}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        unnest $$t <- scan-collection(ordered-list-constructor(ordered-list-constructor(29, cast({ "f1": "a", "f2": 3 })))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- UNNEST  |UNPARTITIONED|
-          empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+    assign [$$21] <- [{"id": get-item($$t, 0), "v": get-item($$t, 1)}] project: [$$21] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      unnest $$t <- scan-collection(ordered-list-constructor(ordered-list-constructor(29, cast({ "f1": "a", "f2": 3 })))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- UNNEST  |UNPARTITIONED|
+        empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.3.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.3.plan
index 3028a95..e24b70a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.3.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.3.plan
@@ -1,14 +1,14 @@
-distribute result [$$15] [cardinality: 0.0, op-cost: 0.0, total-cost: 33.22]
+distribute result [$$15] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
-  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 33.22]
+  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    assign [$$15] <- [true] [cardinality: 0.0, op-cost: 0.0, total-cost: 33.22]
+    assign [$$15] <- [true] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
     -- ASSIGN  |UNPARTITIONED|
-      project ([]) [cardinality: 0.0, op-cost: 0.0, total-cost: 33.22]
+      project ([]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- STREAM_PROJECT  |UNPARTITIONED|
-        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 33.22]
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
         -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-          order (ASC, $$x) [cardinality: 0.0, op-cost: 33.22, total-cost: 33.22]
+          order (ASC, $$x) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- STABLE_SORT [$$x(ASC)]  |UNPARTITIONED|
             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
             -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.5.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.5.plan
index c5d8eb3..2498fdc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.5.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.5.plan
@@ -1,18 +1,16 @@
-distribute result [$$15] [cardinality: 0.0, op-cost: 0.0, total-cost: 33.22]
+distribute result [$$15] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
-  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 33.22]
+  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$15]) [cardinality: 0.0, op-cost: 0.0, total-cost: 33.22]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$15] <- [le($$x, 2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 33.22]
-      -- ASSIGN  |UNPARTITIONED|
-        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 33.22]
-        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-          order (ASC, $$x) [cardinality: 0.0, op-cost: 33.22, total-cost: 33.22]
-          -- STABLE_SORT [$$x(ASC)]  |UNPARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-              unnest $$x <- range(1, 4) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- UNNEST  |UNPARTITIONED|
-                empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+    assign [$$15] <- [le($$x, 2)] project: [$$15] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+        order (ASC, $$x) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$x(ASC)]  |UNPARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+            unnest $$x <- range(1, 4) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- UNNEST  |UNPARTITIONED|
+              empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.6.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.6.plan
index 3cb126f..da04042 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.6.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.6.plan
@@ -1,18 +1,16 @@
-distribute result [$$15] [cardinality: 0.0, op-cost: 0.0, total-cost: 33.22]
+distribute result [$$15] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
-  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 33.22]
+  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$15]) [cardinality: 0.0, op-cost: 0.0, total-cost: 33.22]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$15] <- [or(null, le($$x, 2))] [cardinality: 0.0, op-cost: 0.0, total-cost: 33.22]
-      -- ASSIGN  |UNPARTITIONED|
-        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 33.22]
-        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-          order (ASC, $$x) [cardinality: 0.0, op-cost: 33.22, total-cost: 33.22]
-          -- STABLE_SORT [$$x(ASC)]  |UNPARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-              unnest $$x <- range(1, 4) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- UNNEST  |UNPARTITIONED|
-                empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+    assign [$$15] <- [or(null, le($$x, 2))] project: [$$15] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+        order (ASC, $$x) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$x(ASC)]  |UNPARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+            unnest $$x <- range(1, 4) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- UNNEST  |UNPARTITIONED|
+              empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.7.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.7.plan
index c8c68c9..9ea5c09 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.7.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.7.plan
@@ -1,14 +1,14 @@
-distribute result [$$17] [cardinality: 0.0, op-cost: 0.0, total-cost: 33.22]
+distribute result [$$17] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
-  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 33.22]
+  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    assign [$$17] <- [true] [cardinality: 0.0, op-cost: 0.0, total-cost: 33.22]
+    assign [$$17] <- [true] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
     -- ASSIGN  |UNPARTITIONED|
-      project ([]) [cardinality: 0.0, op-cost: 0.0, total-cost: 33.22]
+      project ([]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- STREAM_PROJECT  |UNPARTITIONED|
-        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 33.22]
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
         -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-          order (ASC, $$x) [cardinality: 0.0, op-cost: 33.22, total-cost: 33.22]
+          order (ASC, $$x) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- STABLE_SORT [$$x(ASC)]  |UNPARTITIONED|
             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
             -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3410/query-ASTERIXDB-3410.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3410/query-ASTERIXDB-3410.3.adm
new file mode 100644
index 0000000..2c44767
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3410/query-ASTERIXDB-3410.3.adm
@@ -0,0 +1,2 @@
+{ "Category 1": 1, "Category 2": 1, "Category 3": 0, "Category 4": 0, "Category 5": 0, "Category 6": 1, "Category 7": 0, "Category 8": 0, "Category 9": 1, "Category 10": 1, "Category 11": 0, "Category 12": 1, "Category 13": 1, "Category 14": 1, "Category 15": 0, "Category 16": 1, "Category 17": 1, "Category 18": 1, "Category 19": 1, "Category 20": 1, "Category 21": 1, "Category 22": 1, "Category 23": 0, "Category 24": 0, "Category 25": 0, "Category 26": 0, "itemid": 10, "description": "ABC" }
+{ "Category 1": 1, "Category 2": 1, "Category 3": 1, "Category 4": 0, "Category 5": 1, "Category 6": 0, "Category 7": 1, "Category 8": 1, "Category 9": 1, "Category 10": 1, "Category 11": 0, "Category 12": 0, "Category 13": 0, "Category 14": 0, "Category 15": 0, "Category 16": 0, "Category 17": 0, "Category 18": 0, "Category 19": 1, "Category 20": 1, "Category 21": 1, "Category 22": 1, "Category 23": 1, "Category 24": 0, "Category 25": 0, "Category 26": 0, "itemid": 12, "description": "XYZ" }
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 34624f9..7978723 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
@@ -2,25 +2,23 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$30]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$30] <- [$$md.getField("name")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$md]) [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 [$$32(ASC) ]  |PARTITIONED|
-            order (ASC, $$32) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                select (neq(uuid(), uuid())) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$32, $$md] <- test.MyDataset [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|
+    assign [$$30] <- [$$md.getField("name")] project: [$$30] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$md]) [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 [$$32(ASC) ]  |PARTITIONED|
+          order (ASC, $$32) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              select (neq(uuid(), uuid())) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$32, $$md] <- test.MyDataset [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/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 45a323c..a539ec9 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
@@ -2,25 +2,23 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$30]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$30] <- [$$md.getField("name")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$md]) [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 [$$32(ASC) ]  |PARTITIONED|
-            order (ASC, $$32) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                select (neq(current-date(), date: { 1980-09-10 })) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$32, $$md] <- test.MyDataset [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|
+    assign [$$30] <- [$$md.getField("name")] project: [$$30] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$md]) [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 [$$32(ASC) ]  |PARTITIONED|
+          order (ASC, $$32) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              select (neq(current-date(), date: { 1980-09-10 })) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$32, $$md] <- test.MyDataset [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/profile/full-scan/full-scan.3.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/full-scan/full-scan.3.regexjson
index 03900f3..68c1311 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/full-scan/full-scan.3.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/full-scan/full-scan.3.regexjson
@@ -24,6 +24,7 @@
                             },
                             "open-time": "R{[0-9]+}",
                             "close-time": "R{[0-9]+}",
+                            "cardinality": "R{[0-9]+}",
                             "offset": "R{[0-9]+}",
                             "frame-times": [
                                 0
@@ -35,7 +36,11 @@
                         {
                             "name": "R{.+}",
                             "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}"
+                            "runtime-id": "R{.+}",
+                            "cardinality-out": "R{[0-9.]+}",
+                            "avg-tuple-size": "R{[0-9.]+}",
+                            "min-tuple-size": "R{[0-9.]+}",
+                            "max-tuple-size": "R{[0-9.]+}"
                         },
                         {
                             "name": "R{.+}",
@@ -86,15 +91,6 @@
                     {
                         "name": "R{.+}",
                         "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}",
-                        "cardinality-out": "R{[0-9.]+}",
-                        "avg-tuple-size": "R{[0-9.]+}",
-                        "min-tuple-size": "R{[0-9.]+}",
-                        "max-tuple-size": "R{[0-9.]+}"
-                    },
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
                         "runtime-id": "R{.+}"
                     }
                   ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/non-unary-subplan/non-unary-subplan.3.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/non-unary-subplan/non-unary-subplan.3.regexjson
index e59f095..3217ebd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/non-unary-subplan/non-unary-subplan.3.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/non-unary-subplan/non-unary-subplan.3.regexjson
@@ -1,241 +1,220 @@
 {
-    "job-id": "R{[A-Z0-9.:]+}",
-    "create-time": "R{[0-9.]+}",
-    "start-time": "R{[0-9.]+}",
-    "queued-time": "R{.+}",
-    "end-time": "R{[0-9.]+}",
-    "counters": [],
-    "joblets": [
+  "job-id": "R{[A-Z0-9.:]+}",
+  "create-time": "R{[0-9.]+}",
+  "start-time": "R{[0-9.]+}",
+  "queued-time": "R{[0-9.]+}",
+  "end-time": "R{[0-9.]+}",
+  "counters": [],
+  "joblets": [
+    {
+      "node-id": "R{.+}",
+      "counters": [],
+      "tasks": [
         {
-            "node-id": "R{.+}",
-            "counters": [],
-            "tasks": [
-                {
-                    "activity-id": "R{[A-Z0-9.:]+}",
-                    "partition": "R{[0-9]+}",
-                    "attempt": "R{[0-9]+}",
-                    "partition-send-profile": [],
-                    "counters": [
-                        {
-                            "name": "R{.+}",
-                            "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}"
-                        }
-                    ]
-                },
-                {
-                    "activity-id": "R{[A-Z0-9.:]+}",
-                    "partition": "R{[0-9]+}",
-                    "attempt": "R{[0-9]+}",
-                    "partition-send-profile": [],
-                    "counters": [
-                        {
-                            "name": "R{ANID:ODID:[0-9]:0\\.1 - MicroOp Subplan(?:.|\n)+}",
-                            "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}",
-                            "cardinality-out": "R{[0-9.]+}",
-                            "avg-tuple-size": "R{[0-9.]+}",
-                            "min-tuple-size": "R{[0-9.]+}",
-                            "max-tuple-size": "R{[0-9.]+}"
-                        },
-                        {
-                            "name": "R{.+}",
-                            "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}",
-                            "cardinality-out": "R{[0-9.]+}",
-                            "avg-tuple-size": "R{[0-9.]+}",
-                            "min-tuple-size": "R{[0-9.]+}",
-                            "max-tuple-size": "R{[0-9.]+}"
-                        },
-                        {
-                            "name": "R{.+}",
-                            "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}",
-                            "cardinality-out": "R{[0-9.]+}",
-                            "avg-tuple-size": "R{[0-9.]+}",
-                            "min-tuple-size": "R{[0-9.]+}",
-                            "max-tuple-size": "R{[0-9.]+}"
-                        },
-                        {
-                            "name": "R{.+}",
-                            "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}",
-                            "cardinality-out": "R{[0-9.]+}",
-                            "avg-tuple-size": "R{[0-9.]+}",
-                            "min-tuple-size": "R{[0-9.]+}",
-                            "max-tuple-size": "R{[0-9.]+}"
-                        },
-                        {
-                            "name": "R{.+}",
-                            "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}",
-                            "cardinality-out": "R{[0-9.]+}",
-                            "avg-tuple-size": "R{[0-9.]+}",
-                            "min-tuple-size": "R{[0-9.]+}",
-                            "max-tuple-size": "R{[0-9.]+}"
-                        },
-                        {
-                            "name": "R{.+}",
-                            "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}",
-                            "cardinality-out": "R{[0-9.]+}",
-                            "avg-tuple-size": "R{[0-9.]+}",
-                            "min-tuple-size": "R{[0-9.]+}",
-                            "max-tuple-size": "R{[0-9.]+}"
-                        },
-                        {
-                            "name": "R{.+}",
-                            "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}",
-                            "cardinality-out": "R{[0-9.]+}",
-                            "avg-tuple-size": "R{[0-9.]+}",
-                            "min-tuple-size": "R{[0-9.]+}",
-                            "max-tuple-size": "R{[0-9.]+}"
-                        },
-                        {
-                            "name": "R{.+}",
-                            "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}",
-                            "cardinality-out": "R{[0-9.]+}",
-                            "avg-tuple-size": "R{[0-9.]+}",
-                            "min-tuple-size": "R{[0-9.]+}",
-                            "max-tuple-size": "R{[0-9.]+}"
-                        },
-                        {
-                            "name": "R{.+}",
-                            "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}",
-                            "cardinality-out": "R{[0-9.]+}",
-                            "avg-tuple-size": "R{[0-9.]+}",
-                            "min-tuple-size": "R{[0-9.]+}",
-                            "max-tuple-size": "R{[0-9.]+}"
-                        },
-                        {
-                            "name": "R{.+}",
-                            "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}",
-                            "cardinality-out": "R{[0-9.]+}",
-                            "avg-tuple-size": "R{[0-9.]+}",
-                            "min-tuple-size": "R{[0-9.]+}",
-                            "max-tuple-size": "R{[0-9.]+}"
-                        },
-                        {
-                            "name": "R{.+}",
-                            "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}",
-                            "cardinality-out": "R{[0-9.]+}",
-                            "avg-tuple-size": "R{[0-9.]+}",
-                            "min-tuple-size": "R{[0-9.]+}",
-                            "max-tuple-size": "R{[0-9.]+}"
-                        },
-                        {
-                            "name": "R{.+}",
-                            "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}",
-                            "cardinality-out": "R{[0-9.]+}",
-                            "avg-tuple-size": "R{[0-9.]+}",
-                            "min-tuple-size": "R{[0-9.]+}",
-                            "max-tuple-size": "R{[0-9.]+}"
-                        },
-                        {
-                            "name": "R{.+}",
-                            "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}",
-                            "cardinality-out": "R{[0-9.]+}",
-                            "avg-tuple-size": "R{[0-9.]+}",
-                            "min-tuple-size": "R{[0-9.]+}",
-                            "max-tuple-size": "R{[0-9.]+}"
-                        },
-                        {
-                            "name": "R{.+}",
-                            "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}",
-                            "cardinality-out": "R{[0-9.]+}",
-                            "avg-tuple-size": "R{[0-9.]+}",
-                            "min-tuple-size": "R{[0-9.]+}",
-                            "max-tuple-size": "R{[0-9.]+}"
-                        },
-                        {
-                            "name": "R{.+}",
-                            "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}",
-                            "cardinality-out": "R{[0-9.]+}",
-                            "avg-tuple-size": "R{[0-9.]+}",
-                            "min-tuple-size": "R{[0-9.]+}",
-                            "max-tuple-size": "R{[0-9.]+}"
-                        },
-                        {
-                            "name": "R{.+}",
-                            "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}"
-                        }
-                    ]
-                },
-                {
-                    "activity-id": "R{[A-Z0-9.:]+}",
-                    "partition": "R{[0-9]+}",
-                    "attempt": "R{[0-9]+}",
-                    "partition-send-profile": [
-                        {
-                            "partition-id": {
-                                "job-id": "R{[A-Z0-9.:]+}",
-                                "connector-id": "R{[A-Z0-9.:]+}",
-                                "sender-index": "R{[0-9]+}",
-                                "receiver-index": "R{[0-9]+}"
-                            },
-                            "open-time": "R{[0-9]+}",
-                            "close-time": "R{[0-9]+}",
-                            "offset": "R{[0-9]+}",
-                            "frame-times": [
-                                0
-                            ],
-                            "resolution": 1
-                        }
-                    ],
-                    "counters": [
-                        {
-                            "name": "R{.+}",
-                            "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}",
-                            "cardinality-out": "R{[0-9.]+}",
-                            "avg-tuple-size": "R{[0-9.]+}",
-                            "min-tuple-size": "R{[0-9.]+}",
-                            "max-tuple-size": "R{[0-9.]+}"
-                        },
-                        {
-                            "name": "R{.+}",
-                            "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}",
-                            "cardinality-out": "R{[0-9.]+}",
-                            "avg-tuple-size": "R{[0-9.]+}",
-                            "min-tuple-size": "R{[0-9.]+}",
-                            "max-tuple-size": "R{[0-9.]+}"
-                        },
-                        {
-                            "name": "R{.+}",
-                            "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}",
-                            "cardinality-out": "R{[0-9.]+}",
-                            "avg-tuple-size": "R{[0-9.]+}",
-                            "min-tuple-size": "R{[0-9.]+}",
-                            "max-tuple-size": "R{[0-9.]+}"
-                        },
-                        {
-                            "name": "R{.+}",
-                            "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}"
-                        },
-                        {
-                            "name": "R{.+}",
-                            "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}",
-                            "cardinality-out": "R{[0-9.]+}",
-                            "avg-tuple-size": "R{[0-9.]+}",
-                            "min-tuple-size": "R{[0-9.]+}",
-                            "max-tuple-size": "R{[0-9.]+}"
-                        }
-                    ]
-                }
-            ]
-    }]
-}
\ No newline at end of file
+          "activity-id": "R{[A-Z0-9.:]+}",
+          "partition": "R{[0-9]+}",
+          "attempt": "R{[0-9]+}",
+          "partition-send-profile": [],
+          "counters": [
+            {
+              "name": "R{(?s).*}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}"
+            }
+          ]
+        },
+        {
+          "activity-id": "R{[A-Z0-9.:]+}",
+          "partition": "R{[0-9]+}",
+          "attempt": "R{[0-9]+}",
+          "partition-send-profile": [],
+          "counters": [
+            {
+              "name": "R{(?s).*}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{(?s).*}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{(?s).*}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{(?s).*}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{(?s).*}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{(?s).*}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{(?s).*}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{(?s).*}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{(?s).*}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{(?s).*}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{(?s).*}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{(?s).*}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{(?s).*}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{(?s).*}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}"
+            }
+          ]
+        },
+        {
+          "activity-id": "R{[A-Z0-9.:]+}",
+          "partition": "R{[0-9]+}",
+          "attempt": "R{[0-9]+}",
+          "partition-send-profile": [
+            {
+              "partition-id": {
+                "job-id": "R{[A-Z0-9.:]+}",
+                "connector-id": "R{[A-Z0-9.:]+}",
+                "sender-index": "R{[0-9]+}",
+                "receiver-index": "R{[0-9]+}"
+              },
+              "open-time": "R{[0-9]+}",
+              "close-time": "R{[0-9]+}",
+              "cardinality": "R{[0-9]+}",
+              "offset": "R{[0-9]+}",
+              "frame-times": [
+                0
+              ],
+              "resolution": 1
+            }
+          ],
+          "counters": [
+            {
+              "name": "R{(?s).*}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{(?s).*}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{(?s).*}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{(?s).*}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            }
+          ]
+        }
+      ]
+    }
+  ]
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/plansleep/sleep.3.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/plansleep/sleep.3.regexjson
index 111ded8..75e1874 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/plansleep/sleep.3.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/plansleep/sleep.3.regexjson
@@ -10,7 +10,11 @@
     "max-time": "R{[0-9.]+}",
     "physical-operator": "DISTRIBUTE_RESULT",
     "execution-mode": "PARTITIONED",
-    "optimizer-estimates": "R{.+}",
+    "optimizer-estimates": {
+      "cardinality": "R{[0-9.]+}",
+      "op-cost": "R{[0-9.]+}",
+      "total-cost": "R{[0-9.]+}"
+    },
     "inputs": [
       {
         "operator": "exchange",
@@ -18,7 +22,11 @@
         "runtime-id": "R{.+}",
         "physical-operator": "ONE_TO_ONE_EXCHANGE",
         "execution-mode": "PARTITIONED",
-        "optimizer-estimates":"R{.+}",
+        "optimizer-estimates": {
+          "cardinality": "R{[0-9.]+}",
+          "op-cost": "R{[0-9.]+}",
+          "total-cost": "R{[0-9.]+}"
+        },
         "inputs": [
           {
             "operator": "project",
@@ -31,9 +39,14 @@
             "max-time": "R{[0-9.]+}",
             "min-cardinality": 3,
             "max-cardinality": 3,
+            "total-cardinality": 3,
             "physical-operator": "STREAM_PROJECT",
             "execution-mode": "PARTITIONED",
-            "optimizer-estimates": "R{.+}",
+            "optimizer-estimates": {
+              "cardinality": "R{[0-9.]+}",
+              "op-cost": "R{[0-9.]+}",
+              "total-cost": "R{[0-9.]+}"
+            },
             "inputs": [
               {
                 "operator": "exchange",
@@ -41,9 +54,16 @@
                 "runtime-id": "R{.+}",
                 "min-time": "R{[0-9.]+}",
                 "max-time": "R{[0-9.]+}",
+                "min-cardinality": "R{[0-9.]+}",
+                "max-cardinality": "R{[0-9.]+}",
+                "total-cardinality": "R{[0-9.]+}",
                 "physical-operator": "SORT_MERGE_EXCHANGE [$$49(ASC) ]",
                 "execution-mode": "PARTITIONED",
-                "optimizer-estimates": "R{.+}",
+                "optimizer-estimates": {
+                  "cardinality": "R{[0-9.]+}",
+                  "op-cost": "R{[0-9.]+}",
+                  "total-cost": "R{[0-9.]+}"
+                },
                 "inputs": [
                   {
                     "operator": "order",
@@ -55,11 +75,30 @@
                     ],
                     "operatorId": "1.5",
                     "runtime-id": "R{.+}",
-                    "min-time": "R{[0-9.]+}",
-                    "max-time": "R{[0-9.]+}",
+                    "activity-stats": [
+                      {
+                        "name": "Sort (Run Generation)",
+                        "id": "ANID:0",
+                        "min-time": "R{[0-9.]+}",
+                        "max-time": "R{[0-9.]+}"
+                      },
+                      {
+                        "name": "Sort (Run Merge)",
+                        "id": "ANID:1",
+                        "min-time": "R{[0-9.]+}",
+                        "max-time": "R{[0-9.]+}",
+                        "min-cardinality": 3,
+                        "max-cardinality": 3,
+                        "total-cardinality": 3
+                      }
+                    ],
                     "physical-operator": "STABLE_SORT [$$49(ASC)]",
                     "execution-mode": "PARTITIONED",
-                    "optimizer-estimates": "R{.+}",
+                    "optimizer-estimates": {
+                      "cardinality": "R{[0-9.]+}",
+                      "op-cost": "R{[0-9.]+}",
+                      "total-cost": "R{[0-9.]+}"
+                    },
                     "inputs": [
                       {
                         "operator": "exchange",
@@ -67,11 +106,21 @@
                         "runtime-id": "R{.+}",
                         "physical-operator": "ONE_TO_ONE_EXCHANGE",
                         "execution-mode": "PARTITIONED",
-                        "optimizer-estimates": "R{.+}",
+                        "optimizer-estimates": {
+                          "cardinality": "R{[0-9.]+}",
+                          "op-cost": "R{[0-9.]+}",
+                          "total-cost": "R{[0-9.]+}"
+                        },
                         "inputs": [
                           {
-                            "operator": "project",
+                            "operator": "assign",
                             "variables": [
+                              "$$49"
+                            ],
+                            "expressions": [
+                              "sleep($$city, 1700)"
+                            ],
+                            "project-variables": [
                               "$$48",
                               "$$49"
                             ],
@@ -81,86 +130,144 @@
                             "max-time": "R{[0-9.]+}",
                             "min-cardinality": 3,
                             "max-cardinality": 3,
-                            "physical-operator": "STREAM_PROJECT",
+                            "total-cardinality": 3,
+                            "physical-operator": "ASSIGN",
                             "execution-mode": "PARTITIONED",
-                            "optimizer-estimates": "R{.+}",
+                            "optimizer-estimates": {
+                              "cardinality": "R{[0-9.]+}",
+                              "op-cost": "R{[0-9.]+}",
+                              "total-cost": "R{[0-9.]+}"
+                            },
                             "inputs": [
                               {
                                 "operator": "assign",
                                 "variables": [
-                                  "$$49"
+                                  "$$48"
                                 ],
                                 "expressions": [
-                                  "sleep($$city, 1700)"
+                                  "{\"customers\": $$52, \"city\": $$city}"
+                                ],
+                                "project-variables": [
+                                  "$$city",
+                                  "$$48"
                                 ],
                                 "operatorId": "1.8",
                                 "runtime-id": "R{.+}",
-                                "min-time": "R{5.+}",
-                                "max-time": "R{5.+}",
+                                "min-time": "R{[0-9.]+}",
+                                "max-time": "R{[0-9.]+}",
                                 "min-cardinality": 3,
                                 "max-cardinality": 3,
+                                "total-cardinality": 3,
                                 "physical-operator": "ASSIGN",
                                 "execution-mode": "PARTITIONED",
-                                "optimizer-estimates": "R{.+}",
+                                "optimizer-estimates": {
+                                  "cardinality": "R{[0-9.]+}",
+                                  "op-cost": "R{[0-9.]+}",
+                                  "total-cost": "R{[0-9.]+}"
+                                },
                                 "inputs": [
                                   {
-                                    "operator": "project",
-                                    "variables": [
-                                      "$$city",
-                                      "$$48"
-                                    ],
+                                    "operator": "exchange",
                                     "operatorId": "1.9",
                                     "runtime-id": "R{.+}",
-                                    "min-time": "R{[0-9.]+}",
-                                    "max-time": "R{[0-9.]+}",
-                                    "min-cardinality": 3,
-                                    "max-cardinality": 3,
-                                    "physical-operator": "STREAM_PROJECT",
+                                    "physical-operator": "ONE_TO_ONE_EXCHANGE",
                                     "execution-mode": "PARTITIONED",
-                                    "optimizer-estimates": "R{.+}",
+                                    "optimizer-estimates": {
+                                      "cardinality": "R{[0-9.]+}",
+                                      "op-cost": "R{[0-9.]+}",
+                                      "total-cost": "R{[0-9.]+}"
+                                    },
                                     "inputs": [
                                       {
-                                        "operator": "assign",
-                                        "variables": [
-                                          "$$48"
+                                        "operator": "group-by",
+                                        "group-by-list": [
+                                          {
+                                            "variable": "$$city",
+                                            "expression": "$$56"
+                                          }
                                         ],
-                                        "expressions": [
-                                          "{\"customers\": $$52, \"city\": $$city}"
+                                        "subplan": [
+                                          {
+                                            "operator": "aggregate",
+                                            "variables": [
+                                              "$$52"
+                                            ],
+                                            "expressions": [
+                                              "agg-sql-sum($$55)"
+                                            ],
+                                            "operatorId": "1.9.1",
+                                            "physical-operator": "AGGREGATE",
+                                            "execution-mode": "LOCAL",
+                                            "inputs": [
+                                              {
+                                                "operator": "nested-tuple-source",
+                                                "operatorId": "1.9.2",
+                                                "physical-operator": "NESTED_TUPLE_SOURCE",
+                                                "execution-mode": "LOCAL"
+                                              }
+                                            ]
+                                          }
                                         ],
                                         "operatorId": "1.10",
                                         "runtime-id": "R{.+}",
-                                        "min-time": "R{[0-9.]+}",
-                                        "max-time": "R{[0-9.]+}",
-                                        "min-cardinality": 3,
-                                        "max-cardinality": 3,
-                                        "physical-operator": "ASSIGN",
+                                        "activity-stats": [
+                                          {
+                                            "name": "Sort (Run Generation)",
+                                            "id": "ANID:0",
+                                            "min-time": "R{[0-9.]+}",
+                                            "max-time": "R{[0-9.]+}"
+                                          },
+                                          {
+                                            "name": "Sort (Run Merge)",
+                                            "id": "ANID:1",
+                                            "min-time": "R{[0-9.]+}",
+                                            "max-time": "R{[0-9.]+}",
+                                            "min-cardinality": 3,
+                                            "max-cardinality": 3,
+                                            "total-cardinality": 3
+                                          }
+                                        ],
+                                        "physical-operator": "SORT_GROUP_BY[$$56]",
                                         "execution-mode": "PARTITIONED",
-                                        "optimizer-estimates": "R{.+}",
+                                        "optimizer-estimates": {
+                                          "cardinality": "R{[0-9.]+}",
+                                          "op-cost": "R{[0-9.]+}",
+                                          "total-cost": "R{[0-9.]+}"
+                                        },
                                         "inputs": [
                                           {
                                             "operator": "exchange",
                                             "operatorId": "1.11",
                                             "runtime-id": "R{.+}",
-                                            "physical-operator": "ONE_TO_ONE_EXCHANGE",
+                                            "min-time": "R{[0-9.]+}",
+                                            "max-time": "R{[0-9.]+}",
+                                            "min-cardinality": 3,
+                                            "max-cardinality": 3,
+                                            "total-cardinality": 3,
+                                            "physical-operator": "HASH_PARTITION_EXCHANGE [$$56]",
                                             "execution-mode": "PARTITIONED",
-                                            "optimizer-estimates": "R{.+}",
+                                            "optimizer-estimates": {
+                                              "cardinality": "R{[0-9.]+}",
+                                              "op-cost": "R{[0-9.]+}",
+                                              "total-cost": "R{[0-9.]+}"
+                                            },
                                             "inputs": [
                                               {
                                                 "operator": "group-by",
                                                 "group-by-list": [
                                                   {
-                                                    "variable": "$$city",
-                                                    "expression": "$$56"
+                                                    "variable": "$$56",
+                                                    "expression": "$$50"
                                                   }
                                                 ],
                                                 "subplan": [
                                                   {
                                                     "operator": "aggregate",
                                                     "variables": [
-                                                      "$$52"
+                                                      "$$55"
                                                     ],
                                                     "expressions": [
-                                                      "agg-sql-sum($$55)"
+                                                      "agg-sql-count(1)"
                                                     ],
                                                     "operatorId": "1.11.1",
                                                     "physical-operator": "AGGREGATE",
@@ -177,177 +284,164 @@
                                                 ],
                                                 "operatorId": "1.12",
                                                 "runtime-id": "R{.+}",
-                                                "min-time": "R{[0-9.]+}",
-                                                "max-time": "R{[0-9.]+}",
-                                                "min-cardinality": 3,
-                                                "max-cardinality": 3,
-                                                "physical-operator": "SORT_GROUP_BY[$$56]",
+                                                "activity-stats": [
+                                                  {
+                                                    "name": "Sort (Run Generation)",
+                                                    "id": "ANID:0",
+                                                    "min-time": "R{[0-9.]+}",
+                                                    "max-time": "R{[0-9.]+}"
+                                                  },
+                                                  {
+                                                    "name": "Sort (Run Merge)",
+                                                    "id": "ANID:1",
+                                                    "min-time": "R{[0-9.]+}",
+                                                    "max-time": "R{[0-9.]+}",
+                                                    "min-cardinality": 3,
+                                                    "max-cardinality": 3,
+                                                    "total-cardinality": 3
+                                                  }
+                                                ],
+                                                "physical-operator": "SORT_GROUP_BY[$$50]",
                                                 "execution-mode": "PARTITIONED",
-                                                "optimizer-estimates": "R{.+}",
+                                                "optimizer-estimates": {
+                                                  "cardinality": "R{[0-9.]+}",
+                                                  "op-cost": "R{[0-9.]+}",
+                                                  "total-cost": "R{[0-9.]+}"
+                                                },
                                                 "inputs": [
                                                   {
                                                     "operator": "exchange",
                                                     "operatorId": "1.13",
                                                     "runtime-id": "R{.+}",
-                                                    "min-time": "R{[0-9.]+}",
-                                                    "max-time": "R{[0-9.]+}",
-                                                    "physical-operator": "HASH_PARTITION_EXCHANGE [$$56]",
+                                                    "physical-operator": "ONE_TO_ONE_EXCHANGE",
                                                     "execution-mode": "PARTITIONED",
-                                                    "optimizer-estimates": "R{.+}",
+                                                    "optimizer-estimates": {
+                                                      "cardinality": "R{[0-9.]+}",
+                                                      "op-cost": "R{[0-9.]+}",
+                                                      "total-cost": "R{[0-9.]+}"
+                                                    },
                                                     "inputs": [
                                                       {
-                                                        "operator": "group-by",
-                                                        "group-by-list": [
-                                                          {
-                                                            "variable": "$$56",
-                                                            "expression": "$$50"
-                                                          }
+                                                        "operator": "assign",
+                                                        "variables": [
+                                                          "$$50"
                                                         ],
-                                                        "subplan": [
-                                                          {
-                                                            "operator": "aggregate",
-                                                            "variables": [
-                                                              "$$55"
-                                                            ],
-                                                            "expressions": [
-                                                              "agg-sql-count(1)"
-                                                            ],
-                                                            "operatorId": "1.13.1",
-                                                            "physical-operator": "AGGREGATE",
-                                                            "execution-mode": "LOCAL",
-                                                            "inputs": [
-                                                              {
-                                                                "operator": "nested-tuple-source",
-                                                                "operatorId": "1.13.2",
-                                                                "physical-operator": "NESTED_TUPLE_SOURCE",
-                                                                "execution-mode": "LOCAL"
-                                                              }
-                                                            ]
-                                                          }
+                                                        "expressions": [
+                                                          "$$c.getField(3).getField(2)"
+                                                        ],
+                                                        "project-variables": [
+                                                          "$$50"
                                                         ],
                                                         "operatorId": "1.14",
                                                         "runtime-id": "R{.+}",
                                                         "min-time": "R{[0-9.]+}",
                                                         "max-time": "R{[0-9.]+}",
-                                                        "physical-operator": "SORT_GROUP_BY[$$50]",
+                                                        "min-cardinality": "R{[0-9.]+}",
+                                                        "max-cardinality": "R{[0-9.]+}",
+                                                        "total-cardinality": "R{[0-9.]+}",
+                                                        "physical-operator": "ASSIGN",
                                                         "execution-mode": "PARTITIONED",
-                                                        "optimizer-estimates": "R{.+}",
+                                                        "optimizer-estimates": {
+                                                          "cardinality": "R{[0-9.]+}",
+                                                          "op-cost": "R{[0-9.]+}",
+                                                          "total-cost": "R{[0-9.]+}"
+                                                        },
                                                         "inputs": [
                                                           {
-                                                            "operator": "exchange",
+                                                            "operator": "select",
+                                                            "condition": "lt($$c.getField(2), 65)",
                                                             "operatorId": "1.15",
                                                             "runtime-id": "R{.+}",
-                                                            "physical-operator": "ONE_TO_ONE_EXCHANGE",
+                                                            "min-time": "R{[0-9.]+}",
+                                                            "max-time": "R{[0-9.]+}",
+                                                            "min-cardinality": 5,
+                                                            "max-cardinality": 5,
+                                                            "total-cardinality": 5,
+                                                            "physical-operator": "STREAM_SELECT",
                                                             "execution-mode": "PARTITIONED",
-                                                            "optimizer-estimates": "R{.+}",
+                                                            "optimizer-estimates": {
+                                                              "cardinality": "R{[0-9.]+}",
+                                                              "op-cost": "R{[0-9.]+}",
+                                                              "total-cost": "R{[0-9.]+}"
+                                                            },
                                                             "inputs": [
                                                               {
                                                                 "operator": "project",
                                                                 "variables": [
-                                                                  "$$50"
+                                                                  "$$c"
                                                                 ],
                                                                 "operatorId": "1.16",
                                                                 "runtime-id": "R{.+}",
                                                                 "min-time": "R{[0-9.]+}",
                                                                 "max-time": "R{[0-9.]+}",
-                                                                "min-cardinality": 5,
-                                                                "max-cardinality": 5,
+                                                                "min-cardinality": 10,
+                                                                "max-cardinality": 10,
+                                                                "total-cardinality": 10,
                                                                 "physical-operator": "STREAM_PROJECT",
                                                                 "execution-mode": "PARTITIONED",
-                                                                "optimizer-estimates": "R{.+}",
+                                                                "optimizer-estimates": {
+                                                                  "cardinality": "R{[0-9.]+}",
+                                                                  "op-cost": "R{[0-9.]+}",
+                                                                  "total-cost": "R{[0-9.]+}"
+                                                                },
                                                                 "inputs": [
                                                                   {
-                                                                    "operator": "assign",
-                                                                    "variables": [
-                                                                      "$$50"
-                                                                    ],
-                                                                    "expressions": [
-                                                                      "$$c.getField(3).getField(2)"
-                                                                    ],
+                                                                    "operator": "exchange",
                                                                     "operatorId": "1.17",
                                                                     "runtime-id": "R{.+}",
-                                                                    "min-time": "R{[0-9.]+}",
-                                                                    "max-time": "R{[0-9.]+}",
-                                                                    "min-cardinality": 5,
-                                                                    "max-cardinality": 5,
-                                                                    "physical-operator": "ASSIGN",
+                                                                    "physical-operator": "ONE_TO_ONE_EXCHANGE",
                                                                     "execution-mode": "PARTITIONED",
-                                                                    "optimizer-estimates": "R{.+}",
+                                                                    "optimizer-estimates": {
+                                                                      "cardinality": "R{[0-9.]+}",
+                                                                      "op-cost": "R{[0-9.]+}",
+                                                                      "total-cost": "R{[0-9.]+}"
+                                                                    },
                                                                     "inputs": [
                                                                       {
-                                                                        "operator": "select",
-                                                                        "condition": "lt($$c.getField(2), 65)",
+                                                                        "operator": "data-scan",
+                                                                        "variables": [
+                                                                          "$$51",
+                                                                          "$$c"
+                                                                        ],
+                                                                        "data-source": "test.Customers",
                                                                         "operatorId": "1.18",
                                                                         "runtime-id": "R{.+}",
                                                                         "min-time": "R{[0-9.]+}",
                                                                         "max-time": "R{[0-9.]+}",
-                                                                        "min-cardinality": 5,
-                                                                        "max-cardinality": 5,
-                                                                        "physical-operator": "STREAM_SELECT",
+                                                                        "min-cardinality": 10,
+                                                                        "max-cardinality": 10,
+                                                                        "total-cardinality": 10,
+                                                                        "physical-operator": "DATASOURCE_SCAN",
                                                                         "execution-mode": "PARTITIONED",
-                                                                        "optimizer-estimates": "R{.+}",
+                                                                        "optimizer-estimates": {
+                                                                          "cardinality": "R{[0-9.]+}",
+                                                                          "op-cost": "R{[0-9.]+}",
+                                                                          "total-cost": "R{[0-9.]+}"
+                                                                        },
                                                                         "inputs": [
                                                                           {
-                                                                            "operator": "project",
-                                                                            "variables": [
-                                                                              "$$c"
-                                                                            ],
+                                                                            "operator": "exchange",
                                                                             "operatorId": "1.19",
                                                                             "runtime-id": "R{.+}",
-                                                                            "min-time": "R{[0-9.]+}",
-                                                                            "max-time": "R{[0-9.]+}",
-                                                                            "min-cardinality": 10,
-                                                                            "max-cardinality": 10,
-                                                                            "physical-operator": "STREAM_PROJECT",
+                                                                            "physical-operator": "ONE_TO_ONE_EXCHANGE",
                                                                             "execution-mode": "PARTITIONED",
-                                                                            "optimizer-estimates": "R{.+}",
+                                                                            "optimizer-estimates": {
+                                                                              "cardinality": "R{[0-9.]+}",
+                                                                              "op-cost": "R{[0-9.]+}",
+                                                                              "total-cost": "R{[0-9.]+}"
+                                                                            },
                                                                             "inputs": [
                                                                               {
-                                                                                "operator": "exchange",
+                                                                                "operator": "empty-tuple-source",
                                                                                 "operatorId": "1.20",
                                                                                 "runtime-id": "R{.+}",
-                                                                                "physical-operator": "ONE_TO_ONE_EXCHANGE",
+                                                                                "physical-operator": "EMPTY_TUPLE_SOURCE",
                                                                                 "execution-mode": "PARTITIONED",
-                                                                                "optimizer-estimates": "R{.+}",
-                                                                                "inputs": [
-                                                                                  {
-                                                                                    "operator": "data-scan",
-                                                                                    "variables": [
-                                                                                      "$$51",
-                                                                                      "$$c"
-                                                                                    ],
-                                                                                    "data-source": "test.Customers",
-                                                                                    "operatorId": "1.21",
-                                                                                    "runtime-id": "R{.+}",
-                                                                                    "min-time": "R{[0-9.]+}",
-                                                                                    "max-time": "R{[0-9.]+}",
-                                                                                    "min-cardinality": 10,
-                                                                                    "max-cardinality": 10,
-                                                                                    "physical-operator": "DATASOURCE_SCAN",
-                                                                                    "execution-mode": "PARTITIONED",
-                                                                                    "optimizer-estimates": "R{.+}",
-                                                                                    "inputs": [
-                                                                                      {
-                                                                                        "operator": "exchange",
-                                                                                        "operatorId": "1.22",
-                                                                                        "runtime-id": "R{.+}",
-                                                                                        "physical-operator": "ONE_TO_ONE_EXCHANGE",
-                                                                                        "execution-mode": "PARTITIONED",
-                                                                                        "optimizer-estimates": "R{.+}",
-                                                                                        "inputs": [
-                                                                                          {
-                                                                                            "operator": "empty-tuple-source",
-                                                                                            "operatorId": "1.23",
-                                                                                            "runtime-id": "R{.+}",
-                                                                                            "physical-operator": "EMPTY_TUPLE_SOURCE",
-                                                                                            "execution-mode": "PARTITIONED",
-                                                                                            "optimizer-estimates": "R{.+}"
-                                                                                          }
-                                                                                        ]
-                                                                                      }
-                                                                                    ]
-                                                                                  }
-                                                                                ]
+                                                                                "optimizer-estimates": {
+                                                                                  "cardinality": "R{[0-9.]+}",
+                                                                                  "op-cost": "R{[0-9.]+}",
+                                                                                  "total-cost": "R{[0-9.]+}"
+                                                                                }
                                                                               }
                                                                             ]
                                                                           }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/sleep/sleep.3.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/sleep/sleep.3.regexjson
index e6d1c0a..b83927a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/sleep/sleep.3.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/sleep/sleep.3.regexjson
@@ -28,11 +28,6 @@
                         {
                             "name": "R{.+}",
                             "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}"
-                        },
-                        {
-                            "name": "R{.+}",
-                            "run-time": "R{[0-9.]+}",
                             "runtime-id": "R{.+}",
                             "pages-read": "R{[0-9.]+}",
                             "pages-read-cold": "R{[0-9.]+}",
@@ -46,9 +41,9 @@
                             "run-time": "R{5.+}",
                             "runtime-id": "R{.+}",
                             "cardinality-out": 10,
-                            "avg-tuple-size": 38,
-                            "min-tuple-size": 38,
-                            "max-tuple-size": 38
+                            "avg-tuple-size": 25,
+                            "min-tuple-size": 25,
+                            "max-tuple-size": 25
                         },
                         {
                             "name": "R{.+}",
@@ -60,13 +55,9 @@
                             "max-tuple-size": "R{[0-9.]+}"
                         },
                         {
-                          "name": "R{.+}",
-                          "run-time": "R{[0-9.]+}",
-                          "runtime-id": "R{.+}",
-                          "cardinality-out": "R{[0-9.]+}",
-                          "avg-tuple-size": "R{[0-9.]+}",
-                          "min-tuple-size": "R{[0-9.]+}",
-                          "max-tuple-size": "R{[0-9.]+}"
+                            "name": "R{.+}",
+                            "run-time": "R{[0-9.]+}",
+                            "runtime-id": "R{.+}"
                         }
                     ]
                 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/sleep/sleep.4.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/sleep/sleep.4.regexjson
index e3e7647..87b6ab1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/sleep/sleep.4.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/sleep/sleep.4.regexjson
@@ -1,166 +1,154 @@
 {
-    "job-id": "R{[A-Z0-9.:]+}",
-    "create-time": "R{[0-9.]+}",
-    "start-time": "R{[0-9.]+}",
-    "queued-time": "R{.+}",
-    "end-time": "R{[0-9.]+}",
-    "counters": [],
-    "joblets": [
+  "job-id": "R{[A-Z0-9.:]+}",
+  "create-time": "R{[0-9.]+}",
+  "start-time": "R{[0-9.]+}",
+  "queued-time": "R{.+}",
+  "end-time": "R{[0-9.]+}",
+  "counters": [],
+  "joblets": [
+    {
+      "node-id": "R{.+}",
+      "counters": [],
+      "tasks": [
         {
-            "node-id": "R{.+}",
-            "counters": [],
-            "tasks": [
-                {
-                    "activity-id": "R{[A-Z0-9.:]+}",
-                    "partition": "R{[0-9]+}",
-                    "attempt": "R{[0-9]+}",
-                    "partition-send-profile": [
-                        {
-                            "partition-id": {
-                                "job-id": "R{[A-Z0-9.:]+}",
-                                "connector-id": "R{[A-Z0-9.:]+}",
-                                "sender-index": "R{[0-9]+}",
-                                "receiver-index": "R{[0-9]+}"
-                            },
-                            "open-time": "R{[0-9]+}",
-                            "close-time": "R{[0-9]+}",
-                            "offset": "R{[0-9]+}",
-                            "frame-times": [
-                                0
-                            ],
-                            "resolution": 1
-                        }
-                    ],
-                    "counters": [
-                        {
-                            "name": "R{.+}",
-                            "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}"
-                        }
-                    ]
-                },
-                {
-                  "activity-id": "R{[A-Z0-9.:]+}",
-                  "partition": "R{[0-9]+}",
-                  "attempt": "R{[0-9]+}",
-                  "partition-send-profile": [],
-                  "counters": [
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}"
-                    }
-                  ]
-                },
-                {
-                  "activity-id": "R{[A-Z0-9.:]+}",
-                  "partition": "R{[0-9]+}",
-                  "attempt": "R{[0-9]+}",
-                  "partition-send-profile": [],
-                  "counters": [
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}",
-                        "cardinality-out": "R{[0-9.]+}",
-                        "avg-tuple-size": "R{[0-9.]+}",
-                        "min-tuple-size": "R{[0-9.]+}",
-                        "max-tuple-size": "R{[0-9.]+}"
-                    },
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}",
-                        "cardinality-out": "R{[0-9.]+}",
-                        "avg-tuple-size": "R{[0-9.]+}",
-                        "min-tuple-size": "R{[0-9.]+}",
-                        "max-tuple-size": "R{[0-9.]+}"
-                    },
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}"
-                    },
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}",
-                        "cardinality-out": "R{[0-9.]+}",
-                        "avg-tuple-size": "R{[0-9.]+}",
-                        "min-tuple-size": "R{[0-9.]+}",
-                        "max-tuple-size": "R{[0-9.]+}"
-                    }
-                  ]
-                },
-                {
-                  "activity-id": "R{[A-Z0-9.:]+}",
-                  "partition": "R{[0-9]+}",
-                  "attempt": "R{[0-9]+}",
-                  "partition-send-profile": [],
-                  "counters": [
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}",
-                        "cardinality-out": 10,
-                        "avg-tuple-size": 140,
-                        "min-tuple-size": 137,
-                        "max-tuple-size": 151
-                    },
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}",
-                        "cardinality-out": 5,
-                        "avg-tuple-size": 145,
-                        "min-tuple-size": 142,
-                        "max-tuple-size": 151
-                    },
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}"
-                    },
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}",
-                        "pages-read": "R{[0-9.]+}",
-                        "pages-read-cold": "R{[0-9.]+}",
-                        "cardinality-out": "R{[0-9.]+}",
-                        "avg-tuple-size": "R{[0-9.]+}",
-                        "min-tuple-size": "R{[0-9.]+}",
-                        "max-tuple-size": "R{[0-9.]+}"
-                    },
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}",
-                        "cardinality-out": 5,
-                        "avg-tuple-size": 16,
-                        "min-tuple-size": 14,
-                        "max-tuple-size": 19
-                    },
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{5.+}",
-                        "runtime-id": "R{.+}",
-                        "cardinality-out": 5,
-                        "avg-tuple-size": 161,
-                        "min-tuple-size": 156,
-                        "max-tuple-size": 170
-                    },
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}",
-                        "cardinality-out": 1,
-                        "avg-tuple-size": 0,
-                        "min-tuple-size": 0,
-                        "max-tuple-size": 0
-                    }
-                  ]
-                }
-            ]
-    }]
+          "activity-id": "R{[A-Z0-9.:]+}",
+          "partition": "R{[0-9]+}",
+          "attempt": "R{[0-9]+}",
+          "partition-send-profile": [
+            {
+              "partition-id": {
+                "job-id": "R{[A-Z0-9.:]+}",
+                "connector-id": "R{[A-Z0-9.:]+}",
+                "sender-index": "R{[0-9]+}",
+                "receiver-index": "R{[0-9]+}"
+              },
+              "open-time": "R{[0-9]+}",
+              "close-time": "R{[0-9]+}",
+              "cardinality": "R{[0-9]+}",
+              "offset": "R{[0-9]+}",
+              "frame-times": [
+                0
+              ],
+              "resolution": 1
+            }
+          ],
+          "counters": [
+            {
+              "name": "R{.+}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            }
+          ]
+        },
+        {
+          "activity-id": "R{[A-Z0-9.:]+}",
+          "partition": "R{[0-9]+}",
+          "attempt": "R{[0-9]+}",
+          "partition-send-profile": [],
+          "counters": [
+            {
+              "name": "R{.+}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}"
+            }
+          ]
+        },
+        {
+          "activity-id": "R{[A-Z0-9.:]+}",
+          "partition": "R{[0-9]+}",
+          "attempt": "R{[0-9]+}",
+          "partition-send-profile": [],
+          "counters": [
+            {
+              "name": "R{.+}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{.+}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{.+}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}"
+            }
+          ]
+        },
+        {
+          "activity-id": "R{[A-Z0-9.:]+}",
+          "partition": "R{[0-9]+}",
+          "attempt": "R{[0-9]+}",
+          "partition-send-profile": [],
+          "counters": [
+            {
+              "name": "R{.+}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": 10,
+              "avg-tuple-size": 140,
+              "min-tuple-size": 137,
+              "max-tuple-size": 151
+            },
+            {
+              "name": "R{.+}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "pages-read": "R{[0-9.]+}",
+              "pages-read-cold": "R{[0-9.]+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{.+}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{.+}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{.+}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{.+}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}"
+            }
+          ]
+        }
+      ]
+    }
+  ]
 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/sleep/sleep.5.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/sleep/sleep.5.regexjson
index 98d7930..187c3c1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/sleep/sleep.5.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/sleep/sleep.5.regexjson
@@ -1,235 +1,219 @@
 {
-    "job-id": "R{[A-Z0-9.:]+}",
-    "create-time": "R{[0-9.]+}",
-    "start-time": "R{[0-9.]+}",
-    "queued-time": "R{.+}",
-    "end-time": "R{[0-9.]+}",
-    "counters": [],
-    "joblets": [
+  "job-id": "R{[A-Z0-9.:]+}",
+  "create-time": "R{[0-9.]+}",
+  "start-time": "R{[0-9.]+}",
+  "queued-time": "R{.+}",
+  "end-time": "R{[0-9.]+}",
+  "counters": [],
+  "joblets": [
+    {
+      "node-id": "R{.+}",
+      "counters": [],
+      "tasks": [
         {
-            "node-id": "R{.+}",
-            "counters": [],
-            "tasks": [
-                {
-                    "activity-id": "R{[A-Z0-9.:]+}",
-                    "partition": "R{[0-9]+}",
-                    "attempt": "R{[0-9]+}",
-                    "partition-send-profile": [
-                        {
-                            "partition-id": {
-                                "job-id": "R{[A-Z0-9.:]+}",
-                                "connector-id": "R{[A-Z0-9.:]+}",
-                                "sender-index": "R{[0-9]+}",
-                                "receiver-index": "R{[0-9]+}"
-                            },
-                            "open-time": "R{[0-9]+}",
-                            "close-time": "R{[0-9]+}",
-                            "offset": "R{[0-9]+}",
-                            "frame-times": [
-                                0
-                            ],
-                            "resolution": 1
-                        }
-                    ],
-                    "counters": [
-                        {
-                            "name": "R{.+}",
-                            "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}"
-                        }
-                    ]
-                },
-                {
-                  "activity-id": "R{[A-Z0-9.:]+}",
-                  "partition": "R{[0-9]+}",
-                  "attempt": "R{[0-9]+}",
-                  "partition-send-profile": [],
-                  "counters": [
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}"
-                    }
-                  ]
-                },
-                {
-                  "activity-id": "R{[A-Z0-9.:]+}",
-                  "partition": "R{[0-9]+}",
-                  "attempt": "R{[0-9]+}",
-                  "partition-send-profile": [],
-                  "counters": [
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{5.+}",
-                        "runtime-id": "R{.+}",
-                        "cardinality-out": 3,
-                        "avg-tuple-size": 75,
-                        "min-tuple-size": 67,
-                        "max-tuple-size": 82
-                    },
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}",
-                        "cardinality-out": "R{[0-9.]+}",
-                        "avg-tuple-size": "R{[0-9.]+}",
-                        "min-tuple-size": "R{[0-9.]+}",
-                        "max-tuple-size": "R{[0-9.]+}"
-                    },
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}",
-                        "cardinality-out": "R{[0-9.]+}",
-                        "avg-tuple-size": "R{[0-9.]+}",
-                        "min-tuple-size": "R{[0-9.]+}",
-                        "max-tuple-size": "R{[0-9.]+}"
-                    },
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}",
-                        "cardinality-out": "R{[0-9.]+}",
-                        "avg-tuple-size": "R{[0-9.]+}",
-                        "min-tuple-size": "R{[0-9.]+}",
-                        "max-tuple-size": "R{[0-9.]+}"
-                    },
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}",
-                        "cardinality-out": "R{[0-9.]+}",
-                        "avg-tuple-size": "R{[0-9.]+}",
-                        "min-tuple-size": "R{[0-9.]+}",
-                        "max-tuple-size": "R{[0-9.]+}"
-                    },
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}"
-                    }
-                  ]
-                },
-                {
-                    "activity-id": "R{[A-Z0-9.:]+}",
-                    "partition": "R{[0-9]+}",
-                    "attempt": "R{[0-9]+}",
-                    "partition-send-profile": [
-                        {
-                            "partition-id": {
-                                "job-id": "R{[A-Z0-9.:]+}",
-                                "connector-id": "R{[A-Z0-9.:]+}",
-                                "sender-index": "R{[0-9]+}",
-                                "receiver-index": "R{[0-9]+}"
-                            },
-                            "open-time": "R{[0-9]+}",
-                            "close-time": "R{[0-9]+}",
-                            "offset": "R{[0-9]+}",
-                            "frame-times": [
-                                0
-                            ],
-                            "resolution": 1
-                        }
-                    ],
-                    "counters": [
-                        {
-                            "name": "R{.+}",
-                            "run-time": "R{[0-9.]+}",
-                            "runtime-id": "R{.+}"
-                        }
-                    ]
-                },
-                {
-                  "activity-id": "R{[A-Z0-9.:]+}",
-                  "partition": "R{[0-9]+}",
-                  "attempt": "R{[0-9]+}",
-                  "partition-send-profile": [],
-                  "counters": [
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}",
-                        "cardinality-out": "R{[0-9.]+}",
-                        "avg-tuple-size": "R{[0-9.]+}",
-                        "min-tuple-size": "R{[0-9.]+}",
-                        "max-tuple-size": "R{[0-9.]+}"
-                    },
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}"
-                    }
-                  ]
-                },
-                {
-                  "activity-id": "R{[A-Z0-9.:]+}",
-                  "partition": "R{[0-9]+}",
-                  "attempt": "R{[0-9]+}",
-                  "partition-send-profile": [],
-                  "counters": [
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}",
-                        "cardinality-out": "R{[0-9.]+}",
-                        "avg-tuple-size": "R{[0-9.]+}",
-                        "min-tuple-size": "R{[0-9.]+}",
-                        "max-tuple-size": "R{[0-9.]+}"
-                    },
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}",
-                        "cardinality-out": "R{[0-9.]+}",
-                        "avg-tuple-size": "R{[0-9.]+}",
-                        "min-tuple-size": "R{[0-9.]+}",
-                        "max-tuple-size": "R{[0-9.]+}"
-                    },
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}",
-                        "cardinality-out": "R{[0-9.]+}",
-                        "avg-tuple-size": "R{[0-9.]+}",
-                        "min-tuple-size": "R{[0-9.]+}",
-                        "max-tuple-size": "R{[0-9.]+}"
-                    },
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}",
-                        "cardinality-out": "R{[0-9.]+}",
-                        "avg-tuple-size": "R{[0-9.]+}",
-                        "min-tuple-size": "R{[0-9.]+}",
-                        "max-tuple-size": "R{[0-9.]+}"
-                    },
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}",
-                        "pages-read": "R{[0-9.]+}",
-                        "pages-read-cold": "R{[0-9.]+}",
-                        "cardinality-out": "R{[0-9.]+}",
-                        "avg-tuple-size": "R{[0-9.]+}",
-                        "min-tuple-size": "R{[0-9.]+}",
-                        "max-tuple-size": "R{[0-9.]+}"
-                    },
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}",
-                        "cardinality-out": "R{[0-9.]+}",
-                        "avg-tuple-size": "R{[0-9.]+}",
-                        "min-tuple-size": "R{[0-9.]+}",
-                        "max-tuple-size": "R{[0-9.]+}"
-                    },
-                    {
-                        "name": "R{.+}",
-                        "run-time": "R{[0-9.]+}",
-                        "runtime-id": "R{.+}"
-                    }
-                  ]
-                }
-            ]
-    }]
+          "activity-id": "R{[A-Z0-9.:]+}",
+          "partition": "R{[0-9]+}",
+          "attempt": "R{[0-9]+}",
+          "partition-send-profile": [
+            {
+              "partition-id": {
+                "job-id": "R{[A-Z0-9.:]+}",
+                "connector-id": "R{[A-Z0-9.:]+}",
+                "sender-index": "R{[0-9]+}",
+                "receiver-index": "R{[0-9]+}"
+              },
+              "open-time": "R{[0-9]+}",
+              "close-time": "R{[0-9]+}",
+              "cardinality": "R{[0-9]+}",
+              "offset": "R{[0-9]+}",
+              "frame-times": [
+                0
+              ],
+              "resolution": 1
+            }
+          ],
+          "counters": [
+            {
+              "name": "R{.+}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            }
+          ]
+        },
+        {
+          "activity-id": "R{[A-Z0-9.:]+}",
+          "partition": "R{[0-9]+}",
+          "attempt": "R{[0-9]+}",
+          "partition-send-profile": [],
+          "counters": [
+            {
+              "name": "R{.+}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}"
+            }
+          ]
+        },
+        {
+          "activity-id": "R{[A-Z0-9.:]+}",
+          "partition": "R{[0-9]+}",
+          "attempt": "R{[0-9]+}",
+          "partition-send-profile": [],
+          "counters": [
+            {
+              "name": "R{.+}",
+              "run-time": "R{5.+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{.+}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{.+}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{.+}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}"
+            }
+          ]
+        },
+        {
+          "activity-id": "R{[A-Z0-9.:]+}",
+          "partition": "R{[0-9]+}",
+          "attempt": "R{[0-9]+}",
+          "partition-send-profile": [
+            {
+              "partition-id": {
+                "job-id": "R{[A-Z0-9.:]+}",
+                "connector-id": "R{[A-Z0-9.:]+}",
+                "sender-index": "R{[0-9]+}",
+                "receiver-index": "R{[0-9]+}"
+              },
+              "open-time": "R{[0-9]+}",
+              "close-time": "R{[0-9]+}",
+              "cardinality": "R{[0-9]+}",
+              "offset": "R{[0-9]+}",
+              "frame-times": [
+                0
+              ],
+              "resolution": 1
+            }
+          ],
+          "counters": [
+            {
+              "name": "R{.+}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            }
+          ]
+        },
+        {
+          "activity-id": "R{[A-Z0-9.:]+}",
+          "partition": "R{[0-9]+}",
+          "attempt": "R{[0-9]+}",
+          "partition-send-profile": [],
+          "counters": [
+            {
+              "name": "R{.+}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{.+}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "pages-read": "R{[0-9.]+}",
+              "pages-read-cold": "R{[0-9.]+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{.+}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{.+}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{.+}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{.+}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}"
+            }
+          ]
+        },
+        {
+          "activity-id": "R{[A-Z0-9.:]+}",
+          "partition": "R{[0-9]+}",
+          "attempt": "R{[0-9]+}",
+          "partition-send-profile": [],
+          "counters": [
+            {
+              "name": "R{.+}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}",
+              "cardinality-out": "R{[0-9.]+}",
+              "avg-tuple-size": "R{[0-9.]+}",
+              "min-tuple-size": "R{[0-9.]+}",
+              "max-tuple-size": "R{[0-9.]+}"
+            },
+            {
+              "name": "R{.+}",
+              "run-time": "R{[0-9.]+}",
+              "runtime-id": "R{.+}"
+            }
+          ]
+        }
+      ]
+    }
+  ]
 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.024.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.024.plan
index a00d7f6..f13eb9e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.024.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.024.plan
@@ -2,19 +2,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$v]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$v] <- [{"SK0": $$14, "PK0": $$15}] [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 [$$15(ASC) ]  |PARTITIONED|
-          order (ASC, $$15) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              data-scan []<-[$$14, $$15] <- test.ds1.ds1_age.query-index [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|
+    assign [$$v] <- [{"SK0": $$14, "PK0": $$15}] project: [$$v] [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 [$$15(ASC) ]  |PARTITIONED|
+        order (ASC, $$15) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            data-scan []<-[$$14, $$15] <- test.ds1.ds1_age.query-index [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/query_index/q01/q01.025.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.025.plan
index 631c16c..2baa9ac 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.025.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.025.plan
@@ -2,19 +2,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$v]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$v] <- [{"SK0": $$14, "SK1": $$15, "PK0": $$16}] [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 [$$16(ASC) ]  |PARTITIONED|
-          order (ASC, $$16) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              data-scan []<-[$$14, $$15, $$16] <- test.ds1.ds1_age_dept.query-index [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|
+    assign [$$v] <- [{"SK0": $$14, "SK1": $$15, "PK0": $$16}] project: [$$v] [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 [$$16(ASC) ]  |PARTITIONED|
+        order (ASC, $$16) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            data-scan []<-[$$14, $$15, $$16] <- test.ds1.ds1_age_dept.query-index [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/query_index/q01/q01.026.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.026.plan
index 9a21589..c8b01e8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.026.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.026.plan
@@ -2,37 +2,35 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$57]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$57] <- [{"age": $$SK0, "dept": $$SK1, "cnt": $$63}] [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 [$$SK1(ASC), $$SK0(ASC) ]  |PARTITIONED|
-          group by ([$$SK1 := $$65; $$SK0 := $$66]) decor ([]) {
-                    aggregate [$$63] <- [agg-sql-sum($$64)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$65, $$66]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$65, $$66]  |PARTITIONED|
-              group by ([$$65 := $$61; $$66 := $$60]) decor ([]) {
-                        aggregate [$$64] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$61, $$60]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$61, $$60]) [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 []<-[$$60, $$61, $$62] <- test.ds1.ds1_age_dept.query-index [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|
+    assign [$$57] <- [{"age": $$SK0, "dept": $$SK1, "cnt": $$63}] project: [$$57] [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 [$$SK1(ASC), $$SK0(ASC) ]  |PARTITIONED|
+        group by ([$$SK1 := $$65; $$SK0 := $$66]) decor ([]) {
+                  aggregate [$$63] <- [agg-sql-sum($$64)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$65, $$66]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$65, $$66]  |PARTITIONED|
+            group by ([$$65 := $$61; $$66 := $$60]) decor ([]) {
+                      aggregate [$$64] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$61, $$60]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                project ([$$61, $$60]) [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 []<-[$$60, $$61, $$62] <- test.ds1.ds1_age_dept.query-index [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/query_index/q01/q01.027.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.027.plan
index 2c28594..f82628d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.027.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.027.plan
@@ -2,21 +2,19 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$33]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$33] <- [{"cnt": $$36}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$36] <- [agg-sql-sum($$37)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$37] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                data-scan []<-[$$34, $$35] <- test.ds1.ds1_dept.query-index [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|
+    assign [$$33] <- [{"cnt": $$36}] project: [$$33] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$36] <- [agg-sql-sum($$37)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$37] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$34, $$35] <- test.ds1.ds1_dept.query-index [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/query_index/q01/q01.028.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.028.plan
index f171976..ee7953e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.028.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.028.plan
@@ -2,37 +2,35 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$45]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$45] <- [{"age": $$SK0, "cnt": $$49}] [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 [$$SK0(ASC) ]  |PARTITIONED|
-          group by ([$$SK0 := $$51]) decor ([]) {
-                    aggregate [$$49] <- [agg-sql-sum($$50)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$51]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$51]  |PARTITIONED|
-              group by ([$$51 := $$47]) decor ([]) {
-                        aggregate [$$50] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- PRE_CLUSTERED_GROUP_BY[$$47]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$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|
-                      data-scan []<-[$$47, $$48] <- test.ds1.ds1_age.query-index [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|
+    assign [$$45] <- [{"age": $$SK0, "cnt": $$49}] project: [$$45] [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 [$$SK0(ASC) ]  |PARTITIONED|
+        group by ([$$SK0 := $$51]) decor ([]) {
+                  aggregate [$$49] <- [agg-sql-sum($$50)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$51]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$51]  |PARTITIONED|
+            group by ([$$51 := $$47]) decor ([]) {
+                      aggregate [$$50] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- PRE_CLUSTERED_GROUP_BY[$$47]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                project ([$$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|
+                    data-scan []<-[$$47, $$48] <- test.ds1.ds1_age.query-index [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/query_index/q01/q01.029.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.029.plan
index ed25d3a..99731ea 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.029.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.029.plan
@@ -2,37 +2,35 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$45]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$45] <- [{"age": $$SK0, "cnt": $$50}] [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 [$$SK0(ASC) ]  |PARTITIONED|
-          group by ([$$SK0 := $$52]) decor ([]) {
-                    aggregate [$$50] <- [agg-sql-sum($$51)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$52]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
-              group by ([$$52 := $$47]) decor ([]) {
-                        aggregate [$$51] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- PRE_CLUSTERED_GROUP_BY[$$47]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$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|
-                      data-scan []<-[$$47, $$48, $$49] <- test.ds1.ds1_age_dept.query-index [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|
+    assign [$$45] <- [{"age": $$SK0, "cnt": $$50}] project: [$$45] [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 [$$SK0(ASC) ]  |PARTITIONED|
+        group by ([$$SK0 := $$52]) decor ([]) {
+                  aggregate [$$50] <- [agg-sql-sum($$51)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$52]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
+            group by ([$$52 := $$47]) decor ([]) {
+                      aggregate [$$51] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- PRE_CLUSTERED_GROUP_BY[$$47]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                project ([$$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|
+                    data-scan []<-[$$47, $$48, $$49] <- test.ds1.ds1_age_dept.query-index [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/query_index/q01/q01.030.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.030.plan
index 05dc8da..7c9818e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.030.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.030.plan
@@ -2,37 +2,35 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$45]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$45] <- [{"age": $$SK1, "cnt": $$50}] [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 [$$SK1(ASC) ]  |PARTITIONED|
-          group by ([$$SK1 := $$52]) decor ([]) {
-                    aggregate [$$50] <- [agg-sql-sum($$51)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$52]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
-              group by ([$$52 := $$48]) decor ([]) {
-                        aggregate [$$51] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$48]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$48]) [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 []<-[$$47, $$48, $$49] <- test.ds1.ds1_dept_age.query-index [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|
+    assign [$$45] <- [{"age": $$SK1, "cnt": $$50}] project: [$$45] [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 [$$SK1(ASC) ]  |PARTITIONED|
+        group by ([$$SK1 := $$52]) decor ([]) {
+                  aggregate [$$50] <- [agg-sql-sum($$51)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$52]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
+            group by ([$$52 := $$48]) decor ([]) {
+                      aggregate [$$51] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$48]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                project ([$$48]) [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 []<-[$$47, $$48, $$49] <- test.ds1.ds1_dept_age.query-index [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/query_index/q01/q01.031.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.031.plan
index 42f5940..8f01ff7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.031.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.031.plan
@@ -2,37 +2,35 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$57]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$57] <- [{"age": $$SK0, "dept": $$SK1, "cnt": $$63}] [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 [$$SK0(ASC), $$SK1(ASC) ]  |PARTITIONED|
-          group by ([$$SK0 := $$65; $$SK1 := $$66]) decor ([]) {
-                    aggregate [$$63] <- [agg-sql-sum($$64)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$65, $$66]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$65, $$66]  |PARTITIONED|
-              group by ([$$65 := $$60; $$66 := $$61]) decor ([]) {
-                        aggregate [$$64] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- PRE_CLUSTERED_GROUP_BY[$$60, $$61]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$60, $$61]) [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 []<-[$$60, $$61, $$62] <- test.ds1.ds1_age_dept.query-index [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|
+    assign [$$57] <- [{"age": $$SK0, "dept": $$SK1, "cnt": $$63}] project: [$$57] [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 [$$SK0(ASC), $$SK1(ASC) ]  |PARTITIONED|
+        group by ([$$SK0 := $$65; $$SK1 := $$66]) decor ([]) {
+                  aggregate [$$63] <- [agg-sql-sum($$64)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$65, $$66]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$65, $$66]  |PARTITIONED|
+            group by ([$$65 := $$60; $$66 := $$61]) decor ([]) {
+                      aggregate [$$64] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- PRE_CLUSTERED_GROUP_BY[$$60, $$61]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                project ([$$60, $$61]) [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 []<-[$$60, $$61, $$62] <- test.ds1.ds1_age_dept.query-index [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/select-star/var_star/var_star.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/var_star/var_star.5.adm
new file mode 100644
index 0000000..6809f72
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/var_star/var_star.5.adm
@@ -0,0 +1,3 @@
+{  }
+{ "c": 2 }
+{  }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/var_star/var_star.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/var_star/var_star.6.adm
new file mode 100644
index 0000000..106bc77
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/var_star/var_star.6.adm
@@ -0,0 +1,3 @@
+{ "b": 1 }
+{ "b": 1, "c": 2 }
+{ "b": 1 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/storage-size/datasource-function/collection-exists/result.040.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/storage-size/datasource-function/collection-exists/result.040.regexjson
new file mode 100644
index 0000000..1e9e352
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/storage-size/datasource-function/collection-exists/result.040.regexjson
@@ -0,0 +1,3 @@
+{
+    "size": "R{.*}"
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/storage-size/datasource-function/external-collection/result.002.ignore b/asterixdb/asterix-app/src/test/resources/runtimets/results/storage-size/datasource-function/external-collection/result.002.ignore
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/storage-size/datasource-function/external-collection/result.002.ignore
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/storage-size/datasource-function/index-exists/result.040.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/storage-size/datasource-function/index-exists/result.040.regexjson
new file mode 100644
index 0000000..1e9e352
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/storage-size/datasource-function/index-exists/result.040.regexjson
@@ -0,0 +1,3 @@
+{
+    "size": "R{.*}"
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/storage-size/datasource-function/sanity/result.040.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/storage-size/datasource-function/sanity/result.040.regexjson
new file mode 100644
index 0000000..5cfd541
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/storage-size/datasource-function/sanity/result.040.regexjson
@@ -0,0 +1 @@
+{"collectionAndIndexesLargerThanCollectionOnly":true,"collectionOnlyLargerThanIndexOnly":true}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-3538/query-ASTERIXDB-3538.3.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-3538/query-ASTERIXDB-3538.3.plan
index 1f2126c..9e9db49 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-3538/query-ASTERIXDB-3538.3.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-3538/query-ASTERIXDB-3538.3.plan
@@ -2,23 +2,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$311]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$311] <- [{"id": $$345}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        select (not(is-null($$345))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- STREAM_SELECT  |PARTITIONED|
-          project ([$$345]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$311] <- [{"id": $$345}] project: [$$311] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (not(is-null($$345))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        assign [$$345] <- [string-default-null($$s.getField("id"))] project: [$$345] [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|
-            assign [$$345] <- [string-default-null($$s.getField("id"))] [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 []<-[$$314, $$s] <- test.dat1 [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|
-                  data-scan []<-[$$314, $$s] <- test.dat1 [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|
+                  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/subquery/query-ASTERIXDB-3538/query-ASTERIXDB-3538.5.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-3538/query-ASTERIXDB-3538.5.plan
index 2f074b7..6f838fa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-3538/query-ASTERIXDB-3538.5.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-3538/query-ASTERIXDB-3538.5.plan
@@ -2,40 +2,34 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$317]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$317] <- [{"dat3": {"id": $$351, "a": string-default-null($$345), "d": int64-default-null($$318)}}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$351, $$345, $$318]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$317] <- [{"dat3": {"id": $$351, "a": string-default-null($$345), "d": int64-default-null($$318)}}] project: [$$317] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (le($$318, get-item($$295, 0))) project: [$$351, $$345, $$318] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$351, $$345, $$318, $$295]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
         -- STREAM_PROJECT  |PARTITIONED|
-          select (le($$318, get-item($$295, 0))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STREAM_SELECT  |PARTITIONED|
-            project ([$$351, $$345, $$318, $$295]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              subplan {
-                        aggregate [$$295] <- [listify($$330)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          aggregate [$$330] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            unnest $$333 <- scan-collection($$343) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SUBPLAN  |PARTITIONED|
-                select (not(is-null($$351))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  project ([$$351, $$345, $$318, $$343]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$351, $$345, $$318, $$343] <- [string-default-null($$s.getField("id")), $$s.getField("a"), $$s.getField("d"), $$s.getField("e")] [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 []<-[$$319, $$s] <- test.dat1 [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|
+          subplan {
+                    aggregate [$$295] <- [listify($$330)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      aggregate [$$330] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        unnest $$333 <- scan-collection($$343) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- SUBPLAN  |PARTITIONED|
+            select (not(is-null($$351))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$351, $$345, $$318, $$343] <- [string-default-null($$s.getField("id")), $$s.getField("a"), $$s.getField("d"), $$s.getField("e")] project: [$$351, $$345, $$318, $$343] [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 []<-[$$319, $$s] <- test.dat1 [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/tpch/query-plans/query-plans.03.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.03.plan
index 1e67e59..b763313 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.03.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.03.plan
@@ -2,43 +2,39 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$134]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$134] <- [{"$1": 1, "l_returnflag": $$l_returnflag, "l_linestatus": $$l_linestatus, "sum_qty": $$140, "sum_base_price": $$141, "sum_disc_price": $$142, "sum_charge": $$143, "avg_qty": $$144, "avg_price": $$145, "avg_disc": $$146, "count_order": $$147}] [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 [$$l_returnflag(ASC), $$l_linestatus(ASC) ]  |PARTITIONED|
-          group by ([$$l_returnflag := $$160; $$l_linestatus := $$161]) decor ([]) {
-                    aggregate [$$140, $$141, $$142, $$143, $$144, $$145, $$146, $$147] <- [agg-global-sql-sum($$152), agg-global-sql-sum($$153), agg-global-sql-sum($$154), agg-global-sql-sum($$155), agg-global-sql-avg($$156), agg-global-sql-avg($$157), agg-global-sql-avg($$158), agg-sql-sum($$159)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$160, $$161]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$160, $$161]  |PARTITIONED|
-              group by ([$$160 := $$135; $$161 := $$136]) decor ([]) {
-                        aggregate [$$152, $$153, $$154, $$155, $$156, $$157, $$158, $$159] <- [agg-local-sql-sum($$88), agg-local-sql-sum($$93), agg-local-sql-sum(numeric-multiply($$93, numeric-subtract(1, $$149))), agg-local-sql-sum(numeric-multiply(numeric-multiply($$93, numeric-subtract(1, $$149)), numeric-add(1, $$151))), agg-local-sql-avg($$88), agg-local-sql-avg($$93), agg-local-sql-avg($$149), agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$135, $$136]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$88, $$93, $$149, $$151, $$135, $$136]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    select (le($$l.getField(10), "1998-09-16")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_SELECT  |PARTITIONED|
-                      assign [$$136, $$135, $$88, $$149, $$151, $$93] <- [$$l.getField(9), $$l.getField(8), $$l.getField(4), $$l.getField(6), $$l.getField(7), $$l.getField(5)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
-                        project ([$$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
+    assign [$$134] <- [{"$1": 1, "l_returnflag": $$l_returnflag, "l_linestatus": $$l_linestatus, "sum_qty": $$140, "sum_base_price": $$141, "sum_disc_price": $$142, "sum_charge": $$143, "avg_qty": $$144, "avg_price": $$145, "avg_disc": $$146, "count_order": $$147}] project: [$$134] [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 [$$l_returnflag(ASC), $$l_linestatus(ASC) ]  |PARTITIONED|
+        group by ([$$l_returnflag := $$160; $$l_linestatus := $$161]) decor ([]) {
+                  aggregate [$$140, $$141, $$142, $$143, $$144, $$145, $$146, $$147] <- [agg-global-sql-sum($$152), agg-global-sql-sum($$153), agg-global-sql-sum($$154), agg-global-sql-sum($$155), agg-global-sql-avg($$156), agg-global-sql-avg($$157), agg-global-sql-avg($$158), agg-sql-sum($$159)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$160, $$161]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$160, $$161]  |PARTITIONED|
+            group by ([$$160 := $$135; $$161 := $$136]) decor ([]) {
+                      aggregate [$$152, $$153, $$154, $$155, $$156, $$157, $$158, $$159] <- [agg-local-sql-sum($$88), agg-local-sql-sum($$93), agg-local-sql-sum(numeric-multiply($$93, numeric-subtract(1, $$149))), agg-local-sql-sum(numeric-multiply(numeric-multiply($$93, numeric-subtract(1, $$149)), numeric-add(1, $$151))), agg-local-sql-avg($$88), agg-local-sql-avg($$93), agg-local-sql-avg($$149), agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$135, $$136]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                select (le($$l.getField(10), "1998-09-16")) project: [$$88, $$93, $$149, $$151, $$135, $$136] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STREAM_SELECT  |PARTITIONED|
+                  assign [$$136, $$135, $$88, $$149, $$151, $$93] <- [$$l.getField(9), $$l.getField(8), $$l.getField(4), $$l.getField(6), $$l.getField(7), $$l.getField(5)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ASSIGN  |PARTITIONED|
+                    project ([$$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 []<-[$$138, $$139, $$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|
-                            data-scan []<-[$$138, $$139, $$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|
-                                empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- EMPTY_TUPLE_SOURCE  |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/tpch/query-plans/query-plans.04.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.04.plan
index 51179bb..7e14303 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.04.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.04.plan
@@ -4,288 +4,264 @@
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit 100 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$194]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$194] <- [{"$1": 2, "s_acctbal": $$237, "s_name": $$239, "n_name": $$238, "p_partkey": $$250, "p_mfgr": $$244, "s_address": $$245, "s_phone": $$246, "s_comment": $$247}] [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 [$$237(DESC), $$238(ASC), $$239(ASC), $$250(ASC) ]  |PARTITIONED|
-            limit 100 [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]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (topK: 100) (DESC, $$237) (ASC, $$238) (ASC, $$239) (ASC, $$250) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STABLE_SORT [topK: 100] [$$237(DESC), $$238(ASC), $$239(ASC), $$250(ASC)]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$237, $$239, $$238, $$250, $$244, $$245, $$246, $$247]) [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|
-                        join (eq($$221, $$206)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HYBRID_HASH_JOIN [$$221][$$206]  |PARTITIONED|
-                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- HASH_PARTITION_EXCHANGE [$$221]  |PARTITIONED|
-                            project ([$$237, $$239, $$238, $$250, $$244, $$245, $$246, $$247, $$221]) [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|
-                                join (eq($$225, $$205)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- HYBRID_HASH_JOIN [$$225][$$205]  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- HASH_PARTITION_EXCHANGE [$$225]  |PARTITIONED|
-                                    project ([$$237, $$239, $$250, $$244, $$245, $$246, $$247, $$225]) [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|
-                                        join (and(eq($$250, $$203), eq($$202, $$231))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- HYBRID_HASH_JOIN [$$250, $$231][$$203, $$202]  |PARTITIONED|
-                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            project ([$$250, $$244, $$231]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              assign [$$231] <- [get-item($$181, 0)] [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]
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  group by ([$$250 := $$200]) decor ([$$244]) {
-                                                            aggregate [$$181] <- [listify($$215)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                            -- AGGREGATE  |LOCAL|
-                                                              aggregate [$$215] <- [agg-sql-min($$208)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                              -- AGGREGATE  |LOCAL|
-                                                                select (not(is-missing($$249))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                -- STREAM_SELECT  |LOCAL|
-                                                                  nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                         } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- PRE_CLUSTERED_GROUP_BY[$$200]  |PARTITIONED|
-                                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      order (ASC, $$200) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- STABLE_SORT [$$200(ASC)]  |PARTITIONED|
-                                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          project ([$$244, $$208, $$249, $$200]) [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 (eq($$207, $$200)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                              -- HYBRID_HASH_JOIN [$$200][$$207]  |PARTITIONED|
-                                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                -- HASH_PARTITION_EXCHANGE [$$200]  |PARTITIONED|
-                                                                  project ([$$244, $$200]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    select (and(eq($$p.getField(5), 37), like($$p.getField(4), "%COPPER"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                                      assign [$$244] <- [$$p.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]
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          data-scan []<-[$$200, $$p] <- tpch.part [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 [$$207]  |PARTITIONED|
-                                                                  assign [$$249] <- [true] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                  -- ASSIGN  |PARTITIONED|
-                                                                    project ([$$208, $$207]) [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|
-                                                                        join (eq($$218, $$213)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                        -- HYBRID_HASH_JOIN [$$218][$$213]  |PARTITIONED|
-                                                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                          -- HASH_PARTITION_EXCHANGE [$$218]  |PARTITIONED|
-                                                                            project ([$$208, $$207, $$218]) [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|
-                                                                                join (eq($$207, $$209)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                -- HYBRID_HASH_JOIN [$$207][$$209]  |PARTITIONED|
-                                                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                  -- HASH_PARTITION_EXCHANGE [$$207]  |PARTITIONED|
-                                                                                    project ([$$207]) [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 []<-[$$207, $$p2] <- tpch.part [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 [$$209]  |PARTITIONED|
-                                                                                    project ([$$208, $$218, $$209]) [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|
-                                                                                        join (eq($$223, $$212)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                        -- HYBRID_HASH_JOIN [$$223][$$212]  |PARTITIONED|
-                                                                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                          -- HASH_PARTITION_EXCHANGE [$$223]  |PARTITIONED|
-                                                                                            project ([$$208, $$209, $$223]) [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|
-                                                                                                join (eq($$211, $$210)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                -- HYBRID_HASH_JOIN [$$210][$$211]  |PARTITIONED|
-                                                                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                    replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                    -- REPLICATE  |PARTITIONED|
-                                                                                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                      -- HASH_PARTITION_EXCHANGE [$$210]  |PARTITIONED|
-                                                                                                        project ([$$208, $$209, $$210]) [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 []<-[$$208, $$209, $$210, $$ps2] <- tpch.partsupp [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 [$$211]  |PARTITIONED|
-                                                                                                    project ([$$223, $$211]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                      assign [$$223] <- [$$s2.getField(3)] [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]
-                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          data-scan []<-[$$211, $$s2] <- tpch.supplier [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 [$$212]  |PARTITIONED|
-                                                                                            project ([$$218, $$212]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                              assign [$$218] <- [$$n2.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]
-                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                  replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                  -- REPLICATE  |PARTITIONED|
-                                                                                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                      data-scan []<-[$$212, $$n2] <- tpch.nation [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  |PARTITIONED|
-                                                                            replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                            -- REPLICATE  |PARTITIONED|
-                                                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                              -- HASH_PARTITION_EXCHANGE [$$213]  |PARTITIONED|
-                                                                                project ([$$213]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                  select (eq($$r2.getField(1), "EUROPE")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                  -- STREAM_SELECT  |PARTITIONED|
-                                                                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      data-scan []<-[$$213, $$r2] <- tpch.region [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 [$$203]  |PARTITIONED|
-                                            project ([$$237, $$239, $$245, $$246, $$247, $$225, $$203, $$202]) [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|
-                                                join (eq($$201, $$204)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- HYBRID_HASH_JOIN [$$201][$$204]  |PARTITIONED|
-                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- HASH_PARTITION_EXCHANGE [$$201]  |PARTITIONED|
-                                                    project ([$$237, $$239, $$245, $$246, $$247, $$225, $$201]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      assign [$$247, $$246, $$245, $$237, $$239, $$225] <- [$$s.getField(6), $$s.getField(4), $$s.getField(2), $$s.getField(5), $$s.getField(1), $$s.getField(3)] [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]
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          data-scan []<-[$$201, $$s] <- tpch.supplier [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  |PARTITIONED|
-                                                    project ([$$202, $$203, $$204]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      assign [$$202, $$203, $$204] <- [$$208, $$209, $$210] [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]
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                          -- REPLICATE  |PARTITIONED|
-                                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                            -- HASH_PARTITION_EXCHANGE [$$210]  |PARTITIONED|
-                                                              project ([$$208, $$209, $$210]) [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 []<-[$$208, $$209, $$210, $$ps2] <- tpch.partsupp [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 [$$205]  |PARTITIONED|
-                                    project ([$$238, $$221, $$205]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      assign [$$238, $$221] <- [$$n.getField(1), $$n.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- ASSIGN  |PARTITIONED|
-                                        project ([$$205, $$n]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          assign [$$205, $$n] <- [$$212, $$n2] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      assign [$$194] <- [{"$1": 2, "s_acctbal": $$237, "s_name": $$239, "n_name": $$238, "p_partkey": $$250, "p_mfgr": $$244, "s_address": $$245, "s_phone": $$246, "s_comment": $$247}] project: [$$194] [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 [$$237(DESC), $$238(ASC), $$239(ASC), $$250(ASC) ]  |PARTITIONED|
+          limit 100 [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]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              order (topK: 100) (DESC, $$237) (ASC, $$238) (ASC, $$239) (ASC, $$250) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STABLE_SORT [topK: 100] [$$237(DESC), $$238(ASC), $$239(ASC), $$250(ASC)]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  project ([$$237, $$239, $$238, $$250, $$244, $$245, $$246, $$247]) [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|
+                      join (eq($$221, $$206)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- HYBRID_HASH_JOIN [$$221][$$206]  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- HASH_PARTITION_EXCHANGE [$$221]  |PARTITIONED|
+                          project ([$$237, $$239, $$238, $$250, $$244, $$245, $$246, $$247, $$221]) [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|
+                              join (eq($$225, $$205)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- HYBRID_HASH_JOIN [$$225][$$205]  |PARTITIONED|
+                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- HASH_PARTITION_EXCHANGE [$$225]  |PARTITIONED|
+                                  project ([$$237, $$239, $$250, $$244, $$245, $$246, $$247, $$225]) [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|
+                                      join (and(eq($$250, $$203), eq($$202, $$231))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- HYBRID_HASH_JOIN [$$250, $$231][$$203, $$202]  |PARTITIONED|
+                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          assign [$$231] <- [get-item($$181, 0)] project: [$$250, $$244, $$231] [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]
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- REPLICATE  |PARTITIONED|
+                                              group by ([$$250 := $$200]) decor ([$$244]) {
+                                                        aggregate [$$181] <- [listify($$215)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                        -- AGGREGATE  |LOCAL|
+                                                          aggregate [$$215] <- [agg-sql-min($$208)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                          -- AGGREGATE  |LOCAL|
+                                                            select (not(is-missing($$249))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                            -- STREAM_SELECT  |LOCAL|
+                                                              nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- PRE_CLUSTERED_GROUP_BY[$$200]  |PARTITIONED|
                                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  data-scan []<-[$$212, $$n2] <- tpch.nation [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                                  order (ASC, $$200) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                  -- STABLE_SORT [$$200(ASC)]  |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  |PARTITIONED|
-                            project ([$$206]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              assign [$$206] <- [$$213] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ASSIGN  |PARTITIONED|
+                                                      project ([$$244, $$208, $$249, $$200]) [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 (eq($$207, $$200)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                          -- HYBRID_HASH_JOIN [$$200][$$207]  |PARTITIONED|
+                                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                            -- HASH_PARTITION_EXCHANGE [$$200]  |PARTITIONED|
+                                                              select (and(eq($$p.getField(5), 37), like($$p.getField(4), "%COPPER"))) project: [$$244, $$200] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                assign [$$244] <- [$$p.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]
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    data-scan []<-[$$200, $$p] <- tpch.part [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 [$$207]  |PARTITIONED|
+                                                              assign [$$249] <- [true] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                project ([$$208, $$207]) [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|
+                                                                    join (eq($$218, $$213)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                    -- HYBRID_HASH_JOIN [$$218][$$213]  |PARTITIONED|
+                                                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                      -- HASH_PARTITION_EXCHANGE [$$218]  |PARTITIONED|
+                                                                        project ([$$208, $$207, $$218]) [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|
+                                                                            join (eq($$207, $$209)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                            -- HYBRID_HASH_JOIN [$$207][$$209]  |PARTITIONED|
+                                                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                              -- HASH_PARTITION_EXCHANGE [$$207]  |PARTITIONED|
+                                                                                project ([$$207]) [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 []<-[$$207, $$p2] <- tpch.part [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 [$$209]  |PARTITIONED|
+                                                                                project ([$$208, $$218, $$209]) [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|
+                                                                                    join (eq($$223, $$212)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                    -- HYBRID_HASH_JOIN [$$223][$$212]  |PARTITIONED|
+                                                                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                      -- HASH_PARTITION_EXCHANGE [$$223]  |PARTITIONED|
+                                                                                        project ([$$208, $$209, $$223]) [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|
+                                                                                            join (eq($$211, $$210)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                            -- HYBRID_HASH_JOIN [$$210][$$211]  |PARTITIONED|
+                                                                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                                  -- HASH_PARTITION_EXCHANGE [$$210]  |PARTITIONED|
+                                                                                                    project ([$$208, $$209, $$210]) [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 []<-[$$208, $$209, $$210, $$ps2] <- tpch.partsupp [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 [$$211]  |PARTITIONED|
+                                                                                                assign [$$223] <- [$$s2.getField(3)] project: [$$223, $$211] [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]
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    data-scan []<-[$$211, $$s2] <- tpch.supplier [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 [$$212]  |PARTITIONED|
+                                                                                        assign [$$218] <- [$$n2.getField(2)] project: [$$218, $$212] [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]
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                data-scan []<-[$$212, $$n2] <- tpch.nation [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  |PARTITIONED|
+                                                                        replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                        -- REPLICATE  |PARTITIONED|
+                                                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                          -- HASH_PARTITION_EXCHANGE [$$213]  |PARTITIONED|
+                                                                            select (eq($$r2.getField(1), "EUROPE")) project: [$$213] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                data-scan []<-[$$213, $$r2] <- tpch.region [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 [$$203]  |PARTITIONED|
+                                          project ([$$237, $$239, $$245, $$246, $$247, $$225, $$203, $$202]) [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|
+                                              join (eq($$201, $$204)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- HYBRID_HASH_JOIN [$$201][$$204]  |PARTITIONED|
+                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                -- HASH_PARTITION_EXCHANGE [$$201]  |PARTITIONED|
+                                                  assign [$$247, $$246, $$245, $$237, $$239, $$225] <- [$$s.getField(6), $$s.getField(4), $$s.getField(2), $$s.getField(5), $$s.getField(1), $$s.getField(3)] project: [$$237, $$239, $$245, $$246, $$247, $$225, $$201] [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]
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      data-scan []<-[$$201, $$s] <- tpch.supplier [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  |PARTITIONED|
+                                                  assign [$$202, $$203, $$204] <- [$$208, $$209, $$210] project: [$$202, $$203, $$204] [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]
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                        -- HASH_PARTITION_EXCHANGE [$$210]  |PARTITIONED|
+                                                          project ([$$208, $$209, $$210]) [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 []<-[$$208, $$209, $$210, $$ps2] <- tpch.partsupp [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  |PARTITIONED|
-                                  replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- REPLICATE  |PARTITIONED|
-                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- HASH_PARTITION_EXCHANGE [$$213]  |PARTITIONED|
-                                      project ([$$213]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        select (eq($$r2.getField(1), "EUROPE")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- STREAM_SELECT  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
+                                  assign [$$238, $$221] <- [$$n.getField(1), $$n.getField(2)] project: [$$238, $$221, $$205] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- ASSIGN  |PARTITIONED|
+                                    assign [$$205, $$n] <- [$$212, $$n2] project: [$$205, $$n] [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]
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- REPLICATE  |PARTITIONED|
                                           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            data-scan []<-[$$213, $$r2] <- tpch.region [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                            data-scan []<-[$$212, $$n2] <- tpch.nation [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  |PARTITIONED|
+                          assign [$$206] <- [$$213] project: [$$206] [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]
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- REPLICATE  |PARTITIONED|
+                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- HASH_PARTITION_EXCHANGE [$$213]  |PARTITIONED|
+                                  select (eq($$r2.getField(1), "EUROPE")) project: [$$213] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      data-scan []<-[$$213, $$r2] <- tpch.region [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/tpch/query-plans/query-plans.05.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.05.plan
index ac4f53f..4cc9d0a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.05.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.05.plan
@@ -4,97 +4,89 @@
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$122]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$122] <- [{"$1": 3, "l_orderkey": $$l_orderkey, "revenue": $$134, "o_orderdate": $$o_orderdate, "o_shippriority": $$o_shippriority}] [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 [$$134(DESC), $$o_orderdate(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]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (topK: 10) (DESC, $$134) (ASC, $$o_orderdate) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STABLE_SORT [topK: 10] [$$134(DESC), $$o_orderdate(ASC)]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    group by ([$$l_orderkey := $$142; $$o_orderdate := $$143; $$o_shippriority := $$144]) decor ([]) {
-                              aggregate [$$134] <- [agg-global-sql-sum($$141)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- AGGREGATE  |LOCAL|
-                                nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                           } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- SORT_GROUP_BY[$$142, $$143, $$144]  |PARTITIONED|
-                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- HASH_PARTITION_EXCHANGE [$$142, $$143, $$144]  |PARTITIONED|
-                        group by ([$$142 := $$131; $$143 := $$127; $$144 := $$125]) decor ([]) {
-                                  aggregate [$$141] <- [agg-local-sql-sum(numeric-multiply($$139, numeric-subtract(1, $$140)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- AGGREGATE  |LOCAL|
-                                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- SORT_GROUP_BY[$$131, $$127, $$125]  |PARTITIONED|
-                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            project ([$$139, $$140, $$131, $$127, $$125]) [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|
-                                join (eq($$131, $$129)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- HYBRID_HASH_JOIN [$$129][$$131]  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- HASH_PARTITION_EXCHANGE [$$129]  |PARTITIONED|
-                                    project ([$$127, $$125, $$129]) [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|
-                                        join (eq($$128, $$136)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- HYBRID_HASH_JOIN [$$128][$$136]  |PARTITIONED|
-                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- HASH_PARTITION_EXCHANGE [$$128]  |PARTITIONED|
-                                            project ([$$128]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              select (eq($$c.getField(6), "BUILDING")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  data-scan []<-[$$128, $$c] <- tpch.customer [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 [$$136]  |PARTITIONED|
-                                            select (lt($$127, "1995-03-22")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- STREAM_SELECT  |PARTITIONED|
-                                              project ([$$129, $$125, $$136, $$127]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                assign [$$125, $$136, $$127] <- [$$o.getField(7), $$o.getField(1), $$o.getField(4)] [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]
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    data-scan []<-[$$129, $$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 [$$131]  |PARTITIONED|
-                                    project ([$$139, $$140, $$131]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      select (gt($$l.getField(10), "1995-03-22")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- STREAM_SELECT  |PARTITIONED|
-                                        assign [$$140, $$139] <- [$$l.getField(6), $$l.getField(5)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- ASSIGN  |PARTITIONED|
-                                          project ([$$131, $$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- STREAM_PROJECT  |PARTITIONED|
+      assign [$$122] <- [{"$1": 3, "l_orderkey": $$l_orderkey, "revenue": $$134, "o_orderdate": $$o_orderdate, "o_shippriority": $$o_shippriority}] project: [$$122] [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 [$$134(DESC), $$o_orderdate(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]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              order (topK: 10) (DESC, $$134) (ASC, $$o_orderdate) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STABLE_SORT [topK: 10] [$$134(DESC), $$o_orderdate(ASC)]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  group by ([$$l_orderkey := $$142; $$o_orderdate := $$143; $$o_shippriority := $$144]) decor ([]) {
+                            aggregate [$$134] <- [agg-global-sql-sum($$141)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- AGGREGATE  |LOCAL|
+                              nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                         } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- SORT_GROUP_BY[$$142, $$143, $$144]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$142, $$143, $$144]  |PARTITIONED|
+                      group by ([$$142 := $$131; $$143 := $$127; $$144 := $$125]) decor ([]) {
+                                aggregate [$$141] <- [agg-local-sql-sum(numeric-multiply($$139, numeric-subtract(1, $$140)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- AGGREGATE  |LOCAL|
+                                  nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                             } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- SORT_GROUP_BY[$$131, $$127, $$125]  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          project ([$$139, $$140, $$131, $$127, $$125]) [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|
+                              join (eq($$131, $$129)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- HYBRID_HASH_JOIN [$$129][$$131]  |PARTITIONED|
+                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- HASH_PARTITION_EXCHANGE [$$129]  |PARTITIONED|
+                                  project ([$$127, $$125, $$129]) [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|
+                                      join (eq($$128, $$136)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- HYBRID_HASH_JOIN [$$128][$$136]  |PARTITIONED|
+                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- HASH_PARTITION_EXCHANGE [$$128]  |PARTITIONED|
+                                          select (eq($$c.getField(6), "BUILDING")) project: [$$128] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- STREAM_SELECT  |PARTITIONED|
                                             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              data-scan []<-[$$130, $$131, $$l] <- tpch.lineitem [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              data-scan []<-[$$128, $$c] <- tpch.customer [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 [$$136]  |PARTITIONED|
+                                          select (lt($$127, "1995-03-22")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            assign [$$125, $$136, $$127] <- [$$o.getField(7), $$o.getField(1), $$o.getField(4)] project: [$$129, $$125, $$136, $$127] [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]
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                data-scan []<-[$$129, $$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 [$$131]  |PARTITIONED|
+                                  select (gt($$l.getField(10), "1995-03-22")) project: [$$139, $$140, $$131] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    assign [$$140, $$139] <- [$$l.getField(6), $$l.getField(5)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- ASSIGN  |PARTITIONED|
+                                      project ([$$131, $$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 []<-[$$130, $$131, $$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|
+                                              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/tpch/query-plans/query-plans.06.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.06.plan
index 2618567..bb627cd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.06.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.06.plan
@@ -2,99 +2,91 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$74]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$74] <- [{"$1": 4, "o_orderpriority": $$o_orderpriority, "order_count": $$81}] [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 [$$o_orderpriority(ASC) ]  |PARTITIONED|
-          group by ([$$o_orderpriority := $$91]) decor ([]) {
-                    aggregate [$$81] <- [agg-sql-sum($$90)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$91]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$91]  |PARTITIONED|
-              group by ([$$91 := $$75]) decor ([]) {
-                        aggregate [$$90] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$75]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$75]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$74] <- [{"$1": 4, "o_orderpriority": $$o_orderpriority, "order_count": $$81}] project: [$$74] [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 [$$o_orderpriority(ASC) ]  |PARTITIONED|
+        group by ([$$o_orderpriority := $$91]) decor ([]) {
+                  aggregate [$$81] <- [agg-sql-sum($$90)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$91]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$91]  |PARTITIONED|
+            group by ([$$91 := $$75]) decor ([]) {
+                      aggregate [$$90] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$75]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                select (neq($$80, 0)) project: [$$75] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STREAM_SELECT  |PARTITIONED|
+                  project ([$$80, $$75]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- STREAM_PROJECT  |PARTITIONED|
-                    select (neq($$80, 0)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_SELECT  |PARTITIONED|
-                      project ([$$80, $$75]) [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|
+                      group by ([$$87 := $$89]) decor ([$$75]) {
+                                aggregate [$$80] <- [agg-sum($$88)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- AGGREGATE  |LOCAL|
+                                  nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                             } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- PRE_CLUSTERED_GROUP_BY[$$89]  |PARTITIONED|
                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          group by ([$$87 := $$89]) decor ([$$75]) {
-                                    aggregate [$$80] <- [agg-sum($$88)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          group by ([$$89 := $$77]) decor ([$$75]) {
+                                    aggregate [$$88] <- [agg-count({"l": $$l})] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                     -- AGGREGATE  |LOCAL|
-                                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      select (not(is-missing($$86))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- STREAM_SELECT  |LOCAL|
+                                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
                                  } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- PRE_CLUSTERED_GROUP_BY[$$89]  |PARTITIONED|
+                          -- PRE_CLUSTERED_GROUP_BY[$$77]  |PARTITIONED|
                             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              group by ([$$89 := $$77]) decor ([$$75]) {
-                                        aggregate [$$88] <- [agg-count({"l": $$l})] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- AGGREGATE  |LOCAL|
-                                          select (not(is-missing($$86))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- STREAM_SELECT  |LOCAL|
-                                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- PRE_CLUSTERED_GROUP_BY[$$77]  |PARTITIONED|
+                              order (ASC, $$77) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- STABLE_SORT [$$77(ASC)]  |PARTITIONED|
                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  order (ASC, $$77) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- STABLE_SORT [$$77(ASC)]  |PARTITIONED|
+                                  project ([$$75, $$l, $$86, $$77]) [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|
-                                      project ([$$75, $$l, $$86, $$77]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                      left outer join (eq($$79, $$77)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- HYBRID_HASH_JOIN [$$77][$$79]  |PARTITIONED|
                                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          left outer join (eq($$79, $$77)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- HYBRID_HASH_JOIN [$$77][$$79]  |PARTITIONED|
-                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- HASH_PARTITION_EXCHANGE [$$77]  |PARTITIONED|
-                                              project ([$$75, $$77]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- HASH_PARTITION_EXCHANGE [$$77]  |PARTITIONED|
+                                          select (and(ge($$76, "1996-05-01"), lt($$76, "1996-08-01"))) project: [$$75, $$77] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            assign [$$75, $$76] <- [$$o.getField(5), $$o.getField(4)] project: [$$77, $$75, $$76] [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]
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                data-scan []<-[$$77, $$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 [$$79]  |PARTITIONED|
+                                          assign [$$86] <- [true] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- ASSIGN  |PARTITIONED|
+                                            select (lt($$l.getField(11), $$l.getField(12))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              project ([$$79, $$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                               -- STREAM_PROJECT  |PARTITIONED|
-                                                select (and(ge($$76, "1996-05-01"), lt($$76, "1996-08-01"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- STREAM_SELECT  |PARTITIONED|
-                                                  project ([$$77, $$75, $$76]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    assign [$$75, $$76] <- [$$o.getField(5), $$o.getField(4)] [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]
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        data-scan []<-[$$77, $$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 [$$79]  |PARTITIONED|
-                                              assign [$$86] <- [true] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- ASSIGN  |PARTITIONED|
-                                                select (lt($$l.getField(11), $$l.getField(12))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- STREAM_SELECT  |PARTITIONED|
-                                                  project ([$$79, $$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 []<-[$$78, $$79, $$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|
-                                                      data-scan []<-[$$78, $$79, $$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|
-                                                          empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                          -- EMPTY_TUPLE_SOURCE  |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/tpch/query-plans/query-plans.07.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.07.plan
index 5e03204..521c4a5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.07.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.07.plan
@@ -2,159 +2,143 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$149]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$149] <- [{"$1": 5, "n_name": $$n_name, "revenue": $$165}] [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 [$$165(DESC) ]  |PARTITIONED|
-          order (DESC, $$165) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$165(DESC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              group by ([$$n_name := $$176]) decor ([]) {
-                        aggregate [$$165] <- [agg-global-sql-sum($$175)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$176]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- HASH_PARTITION_EXCHANGE [$$176]  |PARTITIONED|
-                  group by ([$$176 := $$150]) decor ([]) {
-                            aggregate [$$175] <- [agg-local-sql-sum(numeric-multiply($$173, numeric-subtract(1, $$174)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- AGGREGATE  |LOCAL|
-                              nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                         } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- SORT_GROUP_BY[$$150]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      project ([$$173, $$174, $$150]) [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|
-                          join (eq($$163, $$161)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- HYBRID_HASH_JOIN [$$163][$$161]  |PARTITIONED|
-                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- HASH_PARTITION_EXCHANGE [$$163]  |PARTITIONED|
-                              project ([$$173, $$174, $$150, $$163]) [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|
-                                  join (eq($$154, $$160)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- HYBRID_HASH_JOIN [$$154][$$160]  |PARTITIONED|
-                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- HASH_PARTITION_EXCHANGE [$$154]  |PARTITIONED|
-                                      project ([$$173, $$174, $$154]) [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|
-                                          join (and(eq($$153, $$154), eq($$170, $$159))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- HYBRID_HASH_JOIN [$$153, $$170][$$154, $$159]  |PARTITIONED|
-                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- HASH_PARTITION_EXCHANGE [$$153, $$170]  |PARTITIONED|
-                                              project ([$$173, $$174, $$153, $$170]) [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|
-                                                  join (eq($$158, $$156)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- HYBRID_HASH_JOIN [$$156][$$158]  |PARTITIONED|
-                                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- HASH_PARTITION_EXCHANGE [$$156]  |PARTITIONED|
-                                                      project ([$$153, $$156]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$149] <- [{"$1": 5, "n_name": $$n_name, "revenue": $$165}] project: [$$149] [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 [$$165(DESC) ]  |PARTITIONED|
+        order (DESC, $$165) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$165(DESC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            group by ([$$n_name := $$176]) decor ([]) {
+                      aggregate [$$165] <- [agg-global-sql-sum($$175)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$176]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- HASH_PARTITION_EXCHANGE [$$176]  |PARTITIONED|
+                group by ([$$176 := $$150]) decor ([]) {
+                          aggregate [$$175] <- [agg-local-sql-sum(numeric-multiply($$173, numeric-subtract(1, $$174)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- AGGREGATE  |LOCAL|
+                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- SORT_GROUP_BY[$$150]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    project ([$$173, $$174, $$150]) [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|
+                        join (eq($$163, $$161)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- HYBRID_HASH_JOIN [$$163][$$161]  |PARTITIONED|
+                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- HASH_PARTITION_EXCHANGE [$$163]  |PARTITIONED|
+                            project ([$$173, $$174, $$150, $$163]) [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|
+                                join (eq($$154, $$160)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- HYBRID_HASH_JOIN [$$154][$$160]  |PARTITIONED|
+                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- HASH_PARTITION_EXCHANGE [$$154]  |PARTITIONED|
+                                    project ([$$173, $$174, $$154]) [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|
+                                        join (and(eq($$153, $$154), eq($$170, $$159))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- HYBRID_HASH_JOIN [$$153, $$170][$$154, $$159]  |PARTITIONED|
+                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- HASH_PARTITION_EXCHANGE [$$153, $$170]  |PARTITIONED|
+                                            project ([$$173, $$174, $$153, $$170]) [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|
+                                                join (eq($$158, $$156)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                -- HYBRID_HASH_JOIN [$$156][$$158]  |PARTITIONED|
+                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                  -- HASH_PARTITION_EXCHANGE [$$156]  |PARTITIONED|
+                                                    project ([$$153, $$156]) [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|
+                                                        join (eq($$155, $$167)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                        -- HYBRID_HASH_JOIN [$$155][$$167]  |PARTITIONED|
+                                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                          -- HASH_PARTITION_EXCHANGE [$$155]  |PARTITIONED|
+                                                            assign [$$153] <- [$$c.getField(3)] project: [$$153, $$155] [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]
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                data-scan []<-[$$155, $$c] <- tpch.customer [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 [$$167]  |PARTITIONED|
+                                                            select (and(ge($$151, "1993-01-01"), lt($$151, "1994-01-01"))) project: [$$156, $$167] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                              assign [$$167, $$151] <- [$$o.getField(1), $$o.getField(4)] project: [$$156, $$167, $$151] [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]
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  data-scan []<-[$$156, $$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 [$$158]  |PARTITIONED|
+                                                    assign [$$174, $$173, $$170] <- [$$l.getField(6), $$l.getField(5), $$l.getField(2)] project: [$$173, $$174, $$170, $$158] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      project ([$$158, $$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|
-                                                          join (eq($$155, $$167)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                          -- HYBRID_HASH_JOIN [$$155][$$167]  |PARTITIONED|
-                                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                            -- HASH_PARTITION_EXCHANGE [$$155]  |PARTITIONED|
-                                                              project ([$$153, $$155]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                assign [$$153] <- [$$c.getField(3)] [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]
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    data-scan []<-[$$155, $$c] <- tpch.customer [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 [$$167]  |PARTITIONED|
-                                                              project ([$$156, $$167]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                select (and(ge($$151, "1993-01-01"), lt($$151, "1994-01-01"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                -- STREAM_SELECT  |PARTITIONED|
-                                                                  project ([$$156, $$167, $$151]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    assign [$$167, $$151] <- [$$o.getField(1), $$o.getField(4)] [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]
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        data-scan []<-[$$156, $$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 [$$158]  |PARTITIONED|
-                                                      project ([$$173, $$174, $$170, $$158]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        assign [$$174, $$173, $$170] <- [$$l.getField(6), $$l.getField(5), $$l.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          project ([$$158, $$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                          data-scan []<-[$$157, $$158, $$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|
-                                                              data-scan []<-[$$157, $$158, $$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|
-                                                                  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 [$$154, $$159]  |PARTITIONED|
-                                              project ([$$154, $$159]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                assign [$$154] <- [$$s.getField(3)] [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]
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    data-scan []<-[$$159, $$s] <- tpch.supplier [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 [$$160]  |PARTITIONED|
-                                      project ([$$150, $$163, $$160]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        assign [$$150, $$163] <- [$$n.getField(1), $$n.getField(2)] [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|
                                           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            data-scan []<-[$$160, $$n] <- tpch.nation [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$154, $$159]  |PARTITIONED|
+                                            assign [$$154] <- [$$s.getField(3)] project: [$$154, $$159] [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]
                                               -- 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 [$$161]  |PARTITIONED|
-                              project ([$$161]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                select (eq($$r.getField(1), "AFRICA")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |PARTITIONED|
+                                                data-scan []<-[$$159, $$s] <- tpch.supplier [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  |PARTITIONED|
-                                    data-scan []<-[$$161, $$r] <- tpch.region [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$160]  |PARTITIONED|
+                                    assign [$$150, $$163] <- [$$n.getField(1), $$n.getField(2)] project: [$$150, $$163, $$160] [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]
                                       -- 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 []<-[$$160, $$n] <- tpch.nation [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 [$$161]  |PARTITIONED|
+                            select (eq($$r.getField(1), "AFRICA")) project: [$$161] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- STREAM_SELECT  |PARTITIONED|
+                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                data-scan []<-[$$161, $$r] <- tpch.region [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/tpch/query-plans/query-plans.08.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.08.plan
index c7d3487..31c1460 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.08.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.08.plan
@@ -2,173 +2,155 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$186]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$186] <- [{"$1": 7, "supp_nation": $$supp_nation, "cust_nation": $$cust_nation, "l_year": $$l_year, "revenue": $$200}] [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 [$$supp_nation(ASC), $$cust_nation(ASC), $$l_year(ASC) ]  |PARTITIONED|
-          group by ([$$supp_nation := $$217; $$cust_nation := $$218; $$l_year := $$219]) decor ([]) {
-                    aggregate [$$200] <- [agg-global-sql-sum($$216)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$217, $$218, $$219]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$217, $$218, $$219]  |PARTITIONED|
-              group by ([$$217 := $$191; $$218 := $$192; $$219 := $$214]) decor ([]) {
-                        aggregate [$$216] <- [agg-local-sql-sum($$184)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$191, $$192, $$214]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$184, $$191, $$192, $$214]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$184, $$214] <- [numeric-multiply($$211, numeric-subtract(1, $$212)), get-year(date($$190))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
-                      select (or(and(eq($$191, "KENYA"), eq($$192, "PERU")), and(eq($$191, "PERU"), eq($$192, "KENYA")))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_SELECT  |PARTITIONED|
-                        project ([$$191, $$211, $$212, $$190, $$192]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
+    assign [$$186] <- [{"$1": 7, "supp_nation": $$supp_nation, "cust_nation": $$cust_nation, "l_year": $$l_year, "revenue": $$200}] project: [$$186] [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 [$$supp_nation(ASC), $$cust_nation(ASC), $$l_year(ASC) ]  |PARTITIONED|
+        group by ([$$supp_nation := $$217; $$cust_nation := $$218; $$l_year := $$219]) decor ([]) {
+                  aggregate [$$200] <- [agg-global-sql-sum($$216)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$217, $$218, $$219]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$217, $$218, $$219]  |PARTITIONED|
+            group by ([$$217 := $$191; $$218 := $$192; $$219 := $$214]) decor ([]) {
+                      aggregate [$$216] <- [agg-local-sql-sum($$184)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$191, $$192, $$214]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                assign [$$184, $$214] <- [numeric-multiply($$211, numeric-subtract(1, $$212)), get-year(date($$190))] project: [$$184, $$191, $$192, $$214] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  select (or(and(eq($$191, "KENYA"), eq($$192, "PERU")), and(eq($$191, "PERU"), eq($$192, "KENYA")))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_SELECT  |PARTITIONED|
+                    project ([$$191, $$211, $$212, $$190, $$192]) [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|
+                        join (eq($$201, $$199)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- HYBRID_HASH_JOIN [$$201][$$199]  |PARTITIONED|
                           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            join (eq($$201, $$199)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- HYBRID_HASH_JOIN [$$201][$$199]  |PARTITIONED|
-                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- HASH_PARTITION_EXCHANGE [$$201]  |PARTITIONED|
-                                project ([$$191, $$211, $$212, $$190, $$201]) [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|
-                                    join (eq($$205, $$198)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- HYBRID_HASH_JOIN [$$205][$$198]  |PARTITIONED|
-                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
-                                        project ([$$211, $$212, $$190, $$201, $$205]) [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|
-                                            join (eq($$197, $$210)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- HYBRID_HASH_JOIN [$$210][$$197]  |PARTITIONED|
-                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- HASH_PARTITION_EXCHANGE [$$210]  |PARTITIONED|
-                                                project ([$$211, $$212, $$190, $$205, $$210]) [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|
-                                                    join (eq($$196, $$195)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- HYBRID_HASH_JOIN [$$195][$$196]  |PARTITIONED|
-                                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- HASH_PARTITION_EXCHANGE [$$195]  |PARTITIONED|
-                                                        project ([$$211, $$212, $$190, $$205, $$195]) [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|
-                                                            join (eq($$193, $$204)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                            -- HYBRID_HASH_JOIN [$$193][$$204]  |PARTITIONED|
-                                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                              -- HASH_PARTITION_EXCHANGE [$$193]  |PARTITIONED|
-                                                                project ([$$205, $$193]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  assign [$$205] <- [$$s.getField(3)] [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]
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      data-scan []<-[$$193, $$s] <- tpch.supplier [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 [$$204]  |PARTITIONED|
-                                                                select (and(ge($$190, "1995-01-01"), le($$190, "1996-12-31"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                -- STREAM_SELECT  |PARTITIONED|
-                                                                  project ([$$195, $$212, $$211, $$204, $$190]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    assign [$$212, $$211, $$204, $$190] <- [$$l.getField(6), $$l.getField(5), $$l.getField(2), $$l.getField(10)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                    -- ASSIGN  |PARTITIONED|
-                                                                      project ([$$195, $$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 []<-[$$194, $$195, $$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|
-                                                                              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 [$$196]  |PARTITIONED|
-                                                        project ([$$210, $$196]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          assign [$$210] <- [$$o.getField(1)] [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]
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              data-scan []<-[$$196, $$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 [$$197]  |PARTITIONED|
-                                                project ([$$201, $$197]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  assign [$$201] <- [$$c.getField(3)] [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]
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      data-scan []<-[$$197, $$c] <- tpch.customer [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  |PARTITIONED|
-                                        replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- REPLICATE  |PARTITIONED|
-                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- HASH_PARTITION_EXCHANGE [$$198]  |PARTITIONED|
-                                            project ([$$191, $$198]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              assign [$$191] <- [$$n1.getField(1)] [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]
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  data-scan []<-[$$198, $$n1] <- tpch.nation [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|
+                          -- HASH_PARTITION_EXCHANGE [$$201]  |PARTITIONED|
+                            project ([$$191, $$211, $$212, $$190, $$201]) [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|
-                                project ([$$192, $$199]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  assign [$$192, $$199] <- [$$191, $$198] [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]
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- REPLICATE  |PARTITIONED|
-                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- HASH_PARTITION_EXCHANGE [$$198]  |PARTITIONED|
-                                          project ([$$191, $$198]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            assign [$$191] <- [$$n1.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                join (eq($$205, $$198)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- HYBRID_HASH_JOIN [$$205][$$198]  |PARTITIONED|
+                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
+                                    project ([$$211, $$212, $$190, $$201, $$205]) [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|
+                                        join (eq($$197, $$210)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- HYBRID_HASH_JOIN [$$210][$$197]  |PARTITIONED|
+                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- HASH_PARTITION_EXCHANGE [$$210]  |PARTITIONED|
+                                            project ([$$211, $$212, $$190, $$205, $$210]) [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|
+                                                join (eq($$196, $$195)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                -- HYBRID_HASH_JOIN [$$195][$$196]  |PARTITIONED|
+                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                  -- HASH_PARTITION_EXCHANGE [$$195]  |PARTITIONED|
+                                                    project ([$$211, $$212, $$190, $$205, $$195]) [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|
+                                                        join (eq($$193, $$204)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                        -- HYBRID_HASH_JOIN [$$193][$$204]  |PARTITIONED|
+                                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                          -- HASH_PARTITION_EXCHANGE [$$193]  |PARTITIONED|
+                                                            assign [$$205] <- [$$s.getField(3)] project: [$$205, $$193] [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]
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                data-scan []<-[$$193, $$s] <- tpch.supplier [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 [$$204]  |PARTITIONED|
+                                                            select (and(ge($$190, "1995-01-01"), le($$190, "1996-12-31"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                              assign [$$212, $$211, $$204, $$190] <- [$$l.getField(6), $$l.getField(5), $$l.getField(2), $$l.getField(10)] project: [$$195, $$212, $$211, $$204, $$190] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                project ([$$195, $$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 []<-[$$194, $$195, $$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|
+                                                                        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 [$$196]  |PARTITIONED|
+                                                    assign [$$210] <- [$$o.getField(1)] project: [$$210, $$196] [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]
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        data-scan []<-[$$196, $$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 [$$197]  |PARTITIONED|
+                                            assign [$$201] <- [$$c.getField(3)] project: [$$201, $$197] [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]
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                data-scan []<-[$$198, $$n1] <- tpch.nation [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                data-scan []<-[$$197, $$c] <- tpch.customer [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  |PARTITIONED|
+                                    replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- REPLICATE  |PARTITIONED|
+                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- HASH_PARTITION_EXCHANGE [$$198]  |PARTITIONED|
+                                        assign [$$191] <- [$$n1.getField(1)] project: [$$191, $$198] [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]
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            data-scan []<-[$$198, $$n1] <- tpch.nation [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  |PARTITIONED|
+                            assign [$$192, $$199] <- [$$191, $$198] project: [$$192, $$199] [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]
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- REPLICATE  |PARTITIONED|
+                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- HASH_PARTITION_EXCHANGE [$$198]  |PARTITIONED|
+                                    assign [$$191] <- [$$n1.getField(1)] project: [$$191, $$198] [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]
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        data-scan []<-[$$198, $$n1] <- tpch.nation [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/tpch/query-plans/query-plans.09.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.09.plan
index 7235bcf..0ad796d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.09.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.09.plan
@@ -2,209 +2,187 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$199]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$199] <- [{"$1": 8, "o_year": $$o_year, "mkt_share": numeric-divide($$214, $$215)}] [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]
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          group by ([$$o_year := $$238]) decor ([]) {
-                    aggregate [$$214, $$215] <- [agg-global-sql-sum($$236), agg-global-sql-sum($$237)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$238]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$238]  |PARTITIONED|
-              group by ([$$238 := $$200]) decor ([]) {
-                        aggregate [$$236, $$237] <- [agg-local-sql-sum(switch-case(true, eq($$232, "PERU"), numeric-multiply($$230, numeric-subtract(1, $$231)), 0)), agg-local-sql-sum(numeric-multiply($$230, numeric-subtract(1, $$231)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$200]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$232, $$230, $$231, $$200]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$199] <- [{"$1": 8, "o_year": $$o_year, "mkt_share": numeric-divide($$214, $$215)}] project: [$$199] [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]
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        group by ([$$o_year := $$238]) decor ([]) {
+                  aggregate [$$214, $$215] <- [agg-global-sql-sum($$236), agg-global-sql-sum($$237)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$238]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$238]  |PARTITIONED|
+            group by ([$$238 := $$200]) decor ([]) {
+                      aggregate [$$236, $$237] <- [agg-local-sql-sum(switch-case(true, eq($$232, "PERU"), numeric-multiply($$230, numeric-subtract(1, $$231)), 0)), agg-local-sql-sum(numeric-multiply($$230, numeric-subtract(1, $$231)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$200]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                assign [$$200] <- [get-year(date($$201))] project: [$$232, $$230, $$231, $$200] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$232, $$230, $$231, $$201]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$200] <- [get-year(date($$201))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
-                      project ([$$232, $$230, $$231, $$201]) [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|
+                      join (eq($$212, $$210)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- HYBRID_HASH_JOIN [$$212][$$210]  |PARTITIONED|
                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          join (eq($$212, $$210)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- HYBRID_HASH_JOIN [$$212][$$210]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$212]  |PARTITIONED|
+                          project ([$$232, $$230, $$231, $$201, $$212]) [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]
-                            -- HASH_PARTITION_EXCHANGE [$$212]  |PARTITIONED|
-                              project ([$$232, $$230, $$231, $$201, $$212]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              join (eq($$216, $$209)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- HYBRID_HASH_JOIN [$$216][$$209]  |PARTITIONED|
                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  join (eq($$216, $$209)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- HYBRID_HASH_JOIN [$$216][$$209]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$216]  |PARTITIONED|
+                                  project ([$$230, $$231, $$201, $$212, $$216]) [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]
-                                    -- HASH_PARTITION_EXCHANGE [$$216]  |PARTITIONED|
-                                      project ([$$230, $$231, $$201, $$212, $$216]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      join (eq($$218, $$208)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- HYBRID_HASH_JOIN [$$218][$$208]  |PARTITIONED|
                                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          join (eq($$218, $$208)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- HYBRID_HASH_JOIN [$$218][$$208]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$218]  |PARTITIONED|
+                                          project ([$$230, $$231, $$201, $$216, $$218]) [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]
-                                            -- HASH_PARTITION_EXCHANGE [$$218]  |PARTITIONED|
-                                              project ([$$230, $$231, $$201, $$216, $$218]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              join (eq($$226, $$207)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- HYBRID_HASH_JOIN [$$226][$$207]  |PARTITIONED|
                                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  join (eq($$226, $$207)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- HYBRID_HASH_JOIN [$$226][$$207]  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$226]  |PARTITIONED|
+                                                  project ([$$230, $$231, $$201, $$216, $$226]) [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]
-                                                    -- HASH_PARTITION_EXCHANGE [$$226]  |PARTITIONED|
-                                                      project ([$$230, $$231, $$201, $$216, $$226]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      join (eq($$205, $$206)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                      -- HYBRID_HASH_JOIN [$$205][$$206]  |PARTITIONED|
                                                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          join (eq($$205, $$206)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                          -- HYBRID_HASH_JOIN [$$205][$$206]  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
+                                                          project ([$$230, $$231, $$216, $$205]) [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]
-                                                            -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
-                                                              project ([$$230, $$231, $$216, $$205]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              join (and(eq($$202, $$221), eq($$203, $$223))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                              -- HYBRID_HASH_JOIN [$$202, $$203][$$221, $$223]  |PARTITIONED|
                                                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  join (and(eq($$202, $$221), eq($$203, $$223))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                  -- HYBRID_HASH_JOIN [$$202, $$203][$$221, $$223]  |PARTITIONED|
-                                                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                    -- HASH_PARTITION_EXCHANGE [$$202, $$203]  |PARTITIONED|
-                                                                      join (true) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                      -- NESTED_LOOP  |PARTITIONED|
-                                                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          project ([$$202]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            select (eq($$p.getField(4), "ECONOMY BURNISHED NICKEL")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                            -- STREAM_SELECT  |PARTITIONED|
-                                                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                data-scan []<-[$$202, $$p] <- tpch.part [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]
-                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                          project ([$$216, $$203]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            assign [$$216] <- [$$s.getField(3)] [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]
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                data-scan []<-[$$203, $$s] <- tpch.supplier [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 [$$221, $$223]  |PARTITIONED|
-                                                                      project ([$$230, $$231, $$205, $$221, $$223]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        assign [$$231, $$230, $$223, $$221] <- [$$l.getField(6), $$l.getField(5), $$l.getField(2), $$l.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                        -- ASSIGN  |PARTITIONED|
-                                                                          project ([$$205, $$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 []<-[$$204, $$205, $$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|
-                                                                                  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 [$$206]  |PARTITIONED|
-                                                              select (and(ge($$201, "1995-01-01"), le($$201, "1996-12-31"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                              -- STREAM_SELECT  |PARTITIONED|
-                                                                project ([$$206, $$226, $$201]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  assign [$$226, $$201] <- [$$o.getField(1), $$o.getField(4)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                  -- ASSIGN  |PARTITIONED|
+                                                                -- HASH_PARTITION_EXCHANGE [$$202, $$203]  |PARTITIONED|
+                                                                  join (true) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                  -- NESTED_LOOP  |PARTITIONED|
                                                                     exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      data-scan []<-[$$206, $$o] <- tpch.orders [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                      select (eq($$p.getField(4), "ECONOMY BURNISHED NICKEL")) project: [$$202] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                      -- STREAM_SELECT  |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 [$$207]  |PARTITIONED|
-                                                      project ([$$218, $$207]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        assign [$$218] <- [$$c.getField(3)] [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]
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            data-scan []<-[$$207, $$c] <- tpch.customer [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                          data-scan []<-[$$202, $$p] <- tpch.part [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]
+                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                      assign [$$216] <- [$$s.getField(3)] project: [$$216, $$203] [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]
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          data-scan []<-[$$203, $$s] <- tpch.supplier [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 [$$221, $$223]  |PARTITIONED|
+                                                                  assign [$$231, $$230, $$223, $$221] <- [$$l.getField(6), $$l.getField(5), $$l.getField(2), $$l.getField(1)] project: [$$230, $$231, $$205, $$221, $$223] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    project ([$$205, $$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 []<-[$$204, $$205, $$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|
+                                                                            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 [$$206]  |PARTITIONED|
+                                                          select (and(ge($$201, "1995-01-01"), le($$201, "1996-12-31"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                            assign [$$226, $$201] <- [$$o.getField(1), $$o.getField(4)] project: [$$206, $$226, $$201] [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]
                                                               -- 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 []<-[$$206, $$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 [$$207]  |PARTITIONED|
+                                                  assign [$$218] <- [$$c.getField(3)] project: [$$218, $$207] [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]
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      data-scan []<-[$$207, $$c] <- tpch.customer [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 [$$208]  |PARTITIONED|
+                                          assign [$$212] <- [$$n1.getField(2)] project: [$$212, $$208] [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]
-                                            -- HASH_PARTITION_EXCHANGE [$$208]  |PARTITIONED|
-                                              project ([$$212, $$208]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                assign [$$212] <- [$$n1.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]
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- REPLICATE  |PARTITIONED|
-                                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        data-scan []<-[$$208, $$n1] <- tpch.nation [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 [$$209]  |PARTITIONED|
-                                      project ([$$232, $$209]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        assign [$$232] <- [$$n2.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- ASSIGN  |PARTITIONED|
-                                          project ([$$209, $$n2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            assign [$$209, $$n2] <- [$$208, $$n1] [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]
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- REPLICATE  |PARTITIONED|
-                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    data-scan []<-[$$208, $$n1] <- tpch.nation [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 [$$210]  |PARTITIONED|
-                              project ([$$210]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                select (eq($$r.getField(1), "AMERICA")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    data-scan []<-[$$210, $$r] <- tpch.region [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- REPLICATE  |PARTITIONED|
+                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  data-scan []<-[$$208, $$n1] <- tpch.nation [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 [$$209]  |PARTITIONED|
+                                  assign [$$232] <- [$$n2.getField(1)] project: [$$232, $$209] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- ASSIGN  |PARTITIONED|
+                                    assign [$$209, $$n2] <- [$$208, $$n1] project: [$$209, $$n2] [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]
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- REPLICATE  |PARTITIONED|
+                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            data-scan []<-[$$208, $$n1] <- tpch.nation [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 [$$210]  |PARTITIONED|
+                          select (eq($$r.getField(1), "AMERICA")) project: [$$210] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- STREAM_SELECT  |PARTITIONED|
+                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              data-scan []<-[$$210, $$r] <- tpch.region [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/tpch/query-plans/query-plans.10.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.10.plan
index e81b59a..06b0977 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.10.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.10.plan
@@ -2,125 +2,117 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$122]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$122] <- [{"$1": 9, "$2": $$140}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$140] <- [agg-sql-sum($$144)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$144] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                join (eq($$138, $$137)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- HYBRID_HASH_JOIN [$$138][$$137]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HASH_PARTITION_EXCHANGE [$$138]  |PARTITIONED|
-                    project ([$$138]) [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|
-                        join (eq($$136, $$132)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HYBRID_HASH_JOIN [$$132][$$136]  |PARTITIONED|
-                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- HASH_PARTITION_EXCHANGE [$$132]  |PARTITIONED|
-                            project ([$$138, $$132]) [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|
-                                join (and(eq($$130, $$135), eq($$134, $$125))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- HYBRID_HASH_JOIN [$$130, $$125][$$135, $$134]  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    project ([$$130, $$138, $$132, $$125]) [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|
-                                        join (and(eq($$130, $$128), eq($$129, $$125))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- HYBRID_HASH_JOIN [$$130, $$129][$$128, $$125]  |PARTITIONED|
-                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- HASH_PARTITION_EXCHANGE [$$130, $$129]  |PARTITIONED|
-                                            join (true) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- NESTED_LOOP  |PARTITIONED|
+    assign [$$122] <- [{"$1": 9, "$2": $$140}] project: [$$122] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$140] <- [agg-sql-sum($$144)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$144] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              join (eq($$138, $$137)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- HYBRID_HASH_JOIN [$$138][$$137]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HASH_PARTITION_EXCHANGE [$$138]  |PARTITIONED|
+                  project ([$$138]) [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|
+                      join (eq($$136, $$132)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- HYBRID_HASH_JOIN [$$132][$$136]  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- HASH_PARTITION_EXCHANGE [$$132]  |PARTITIONED|
+                          project ([$$138, $$132]) [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|
+                              join (and(eq($$130, $$135), eq($$134, $$125))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- HYBRID_HASH_JOIN [$$130, $$125][$$135, $$134]  |PARTITIONED|
+                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  project ([$$130, $$138, $$132, $$125]) [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|
+                                      join (and(eq($$130, $$128), eq($$129, $$125))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- HYBRID_HASH_JOIN [$$130, $$129][$$128, $$125]  |PARTITIONED|
+                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- HASH_PARTITION_EXCHANGE [$$130, $$129]  |PARTITIONED|
+                                          join (true) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- NESTED_LOOP  |PARTITIONED|
+                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              select (like($$p.getField(1), "%green%")) project: [$$129] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  data-scan []<-[$$129, $$p] <- tpch.part [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]
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              assign [$$138] <- [$$s.getField(3)] project: [$$130, $$138] [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]
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  data-scan []<-[$$130, $$s] <- tpch.supplier [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 [$$128, $$125]  |PARTITIONED|
+                                          assign [$$128, $$125] <- [$$l.getField(2), $$l.getField(1)] project: [$$132, $$128, $$125] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- ASSIGN  |PARTITIONED|
+                                            project ([$$132, $$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|
-                                                project ([$$129]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  select (like($$p.getField(1), "%green%")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- STREAM_SELECT  |PARTITIONED|
-                                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      data-scan []<-[$$129, $$p] <- tpch.part [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]
-                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                project ([$$130, $$138]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  assign [$$138] <- [$$s.getField(3)] [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]
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      data-scan []<-[$$130, $$s] <- tpch.supplier [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 [$$128, $$125]  |PARTITIONED|
-                                            project ([$$132, $$128, $$125]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              assign [$$128, $$125] <- [$$l.getField(2), $$l.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- ASSIGN  |PARTITIONED|
-                                                project ([$$132, $$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                data-scan []<-[$$131, $$132, $$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|
-                                                    data-scan []<-[$$131, $$132, $$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|
-                                                        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 [$$135, $$134]  |PARTITIONED|
-                                    project ([$$134, $$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 []<-[$$133, $$134, $$135, $$ps] <- tpch.partsupp [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 [$$136]  |PARTITIONED|
-                            project ([$$136]) [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 []<-[$$136, $$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 [$$137]  |PARTITIONED|
-                    project ([$$137]) [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 []<-[$$137, $$n] <- tpch.nation [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|
+                                                    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 [$$135, $$134]  |PARTITIONED|
+                                  project ([$$134, $$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 []<-[$$133, $$134, $$135, $$ps] <- tpch.partsupp [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 [$$136]  |PARTITIONED|
+                          project ([$$136]) [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 []<-[$$136, $$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 [$$137]  |PARTITIONED|
+                  project ([$$137]) [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 []<-[$$137, $$n] <- tpch.nation [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/tpch/query-plans/query-plans.11.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.11.plan
index f1dbb5b..e0ee9d6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.11.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-plans/query-plans.11.plan
@@ -4,121 +4,109 @@
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit 20 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$185]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$185] <- [{"$1": 10, "c_custkey": $$c_custkey, "c_name": $$c_name, "revenue": $$203, "c_acctbal": $$c_acctbal, "n_name": $$n_name, "c_address": $$c_address, "c_phone": $$c_phone, "c_comment": $$c_comment}] [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 [$$203(DESC) ]  |PARTITIONED|
-            limit 20 [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]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (topK: 20) (DESC, $$203) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STABLE_SORT [topK: 20] [$$203(DESC)]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    group by ([$$c_custkey := $$211; $$n_name := $$215]) decor ([$$c_name := $$212; $$c_acctbal := $$213; $$c_phone := $$214; $$c_address := $$216; $$c_comment := $$217]) {
-                              aggregate [$$203] <- [agg-global-sql-sum($$210)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- AGGREGATE  |LOCAL|
-                                nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                           } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- SORT_GROUP_BY[$$211, $$215]  |PARTITIONED|
-                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- HASH_PARTITION_EXCHANGE [$$211, $$215]  |PARTITIONED|
-                        group by ([$$211 := $$195; $$215 := $$190]) decor ([$$212 := $$187; $$213 := $$188; $$214 := $$189; $$216 := $$191; $$217 := $$192]) {
-                                  aggregate [$$210] <- [agg-local-sql-sum(numeric-multiply($$208, numeric-subtract(1, $$209)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- AGGREGATE  |LOCAL|
-                                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- SORT_GROUP_BY[$$195, $$190]  |PARTITIONED|
-                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            project ([$$208, $$209, $$195, $$187, $$188, $$189, $$190, $$191, $$192]) [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|
-                                join (eq($$201, $$199)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- HYBRID_HASH_JOIN [$$201][$$199]  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- HASH_PARTITION_EXCHANGE [$$201]  |PARTITIONED|
-                                    project ([$$208, $$209, $$195, $$187, $$188, $$189, $$191, $$192, $$201]) [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|
-                                        join (eq($$198, $$196)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- HYBRID_HASH_JOIN [$$196][$$198]  |PARTITIONED|
-                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- HASH_PARTITION_EXCHANGE [$$196]  |PARTITIONED|
-                                            project ([$$195, $$187, $$188, $$189, $$191, $$192, $$201, $$196]) [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|
-                                                join (eq($$195, $$206)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- HYBRID_HASH_JOIN [$$195][$$206]  |PARTITIONED|
-                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- HASH_PARTITION_EXCHANGE [$$195]  |PARTITIONED|
-                                                    project ([$$195, $$187, $$188, $$189, $$191, $$192, $$201]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      assign [$$192, $$191, $$189, $$188, $$187, $$201] <- [$$c.getField(7), $$c.getField(2), $$c.getField(4), $$c.getField(5), $$c.getField(1), $$c.getField(3)] [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]
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          data-scan []<-[$$195, $$c] <- tpch.customer [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 [$$206]  |PARTITIONED|
-                                                    project ([$$196, $$206]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      select (and(lt($$193, "1993-10-01"), ge($$193, "1993-07-01"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                        project ([$$196, $$206, $$193]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          assign [$$206, $$193] <- [$$o.getField(1), $$o.getField(4)] [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]
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              data-scan []<-[$$196, $$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 [$$198]  |PARTITIONED|
-                                            project ([$$208, $$209, $$198]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              select (eq($$l.getField(8), "R")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                assign [$$209, $$208] <- [$$l.getField(6), $$l.getField(5)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- ASSIGN  |PARTITIONED|
-                                                  project ([$$198, $$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- STREAM_PROJECT  |PARTITIONED|
+      assign [$$185] <- [{"$1": 10, "c_custkey": $$c_custkey, "c_name": $$c_name, "revenue": $$203, "c_acctbal": $$c_acctbal, "n_name": $$n_name, "c_address": $$c_address, "c_phone": $$c_phone, "c_comment": $$c_comment}] project: [$$185] [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 [$$203(DESC) ]  |PARTITIONED|
+          limit 20 [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]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              order (topK: 20) (DESC, $$203) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STABLE_SORT [topK: 20] [$$203(DESC)]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  group by ([$$c_custkey := $$211; $$n_name := $$215]) decor ([$$c_name := $$212; $$c_acctbal := $$213; $$c_phone := $$214; $$c_address := $$216; $$c_comment := $$217]) {
+                            aggregate [$$203] <- [agg-global-sql-sum($$210)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- AGGREGATE  |LOCAL|
+                              nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                         } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- SORT_GROUP_BY[$$211, $$215]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$211, $$215]  |PARTITIONED|
+                      group by ([$$211 := $$195; $$215 := $$190]) decor ([$$212 := $$187; $$213 := $$188; $$214 := $$189; $$216 := $$191; $$217 := $$192]) {
+                                aggregate [$$210] <- [agg-local-sql-sum(numeric-multiply($$208, numeric-subtract(1, $$209)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- AGGREGATE  |LOCAL|
+                                  nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                             } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- SORT_GROUP_BY[$$195, $$190]  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          project ([$$208, $$209, $$195, $$187, $$188, $$189, $$190, $$191, $$192]) [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|
+                              join (eq($$201, $$199)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- HYBRID_HASH_JOIN [$$201][$$199]  |PARTITIONED|
+                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- HASH_PARTITION_EXCHANGE [$$201]  |PARTITIONED|
+                                  project ([$$208, $$209, $$195, $$187, $$188, $$189, $$191, $$192, $$201]) [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|
+                                      join (eq($$198, $$196)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- HYBRID_HASH_JOIN [$$196][$$198]  |PARTITIONED|
+                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- HASH_PARTITION_EXCHANGE [$$196]  |PARTITIONED|
+                                          project ([$$195, $$187, $$188, $$189, $$191, $$192, $$201, $$196]) [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|
+                                              join (eq($$195, $$206)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- HYBRID_HASH_JOIN [$$195][$$206]  |PARTITIONED|
+                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                -- HASH_PARTITION_EXCHANGE [$$195]  |PARTITIONED|
+                                                  assign [$$192, $$191, $$189, $$188, $$187, $$201] <- [$$c.getField(7), $$c.getField(2), $$c.getField(4), $$c.getField(5), $$c.getField(1), $$c.getField(3)] project: [$$195, $$187, $$188, $$189, $$191, $$192, $$201] [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]
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      data-scan []<-[$$197, $$198, $$l] <- tpch.lineitem [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                      data-scan []<-[$$195, $$c] <- tpch.customer [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 [$$199]  |PARTITIONED|
-                                    project ([$$190, $$199]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      assign [$$190] <- [$$n.getField(1)] [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]
+                                                -- HASH_PARTITION_EXCHANGE [$$206]  |PARTITIONED|
+                                                  select (and(lt($$193, "1993-10-01"), ge($$193, "1993-07-01"))) project: [$$196, $$206] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                    assign [$$206, $$193] <- [$$o.getField(1), $$o.getField(4)] project: [$$196, $$206, $$193] [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]
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        data-scan []<-[$$196, $$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 [$$198]  |PARTITIONED|
+                                          select (eq($$l.getField(8), "R")) project: [$$208, $$209, $$198] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            assign [$$209, $$208] <- [$$l.getField(6), $$l.getField(5)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                            -- ASSIGN  |PARTITIONED|
+                                              project ([$$198, $$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 []<-[$$197, $$198, $$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|
+                                                      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 [$$199]  |PARTITIONED|
+                                  assign [$$190] <- [$$n.getField(1)] project: [$$190, $$199] [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]
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      data-scan []<-[$$199, $$n] <- tpch.nation [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|
-                                          data-scan []<-[$$199, $$n] <- tpch.nation [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|
+                                          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/union/union_opt_1/union_opt_1.11.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.11.plan
index 3b08d56..8e75d32 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.11.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.11.plan
@@ -14,93 +14,77 @@
             -- UNION_ALL  |PARTITIONED|
               exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
               -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                project ([$$223]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$223] <- [{"two": $$193}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ASSIGN  |PARTITIONED|
-                    limit 4 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_LIMIT  |PARTITIONED|
-                      project ([$$193]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                assign [$$223] <- [{"two": $$193}] project: [$$223] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  limit 4 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_LIMIT  |PARTITIONED|
+                    assign [$$193] <- [$$onek1.getField(2)] project: [$$193] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ASSIGN  |PARTITIONED|
+                      project ([$$onek1]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$193] <- [$$onek1.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ASSIGN  |PARTITIONED|
-                          project ([$$onek1]) [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 []<-[$$197, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), 1), le($$onek1.getField(2), 10))) limit 4 [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|
-                              data-scan []<-[$$197, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), 1), le($$onek1.getField(2), 10))) limit 4 [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|
+                              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_PARTITION_EXCHANGE  |PARTITIONED|
-                project ([$$237]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$237] <- [{"two": $$194}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ASSIGN  |PARTITIONED|
-                    limit 4 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_LIMIT  |PARTITIONED|
-                      project ([$$194]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                assign [$$237] <- [{"two": $$194}] project: [$$237] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  limit 4 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_LIMIT  |PARTITIONED|
+                    assign [$$194] <- [$$onek2.getField(2)] project: [$$194] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ASSIGN  |PARTITIONED|
+                      project ([$$onek2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$194] <- [$$onek2.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ASSIGN  |PARTITIONED|
-                          project ([$$onek2]) [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 []<-[$$198, $$onek2] <- test.onek2 condition (and(ge($$onek2.getField(2), 1), le($$onek2.getField(2), 100))) limit 4 [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|
-                              data-scan []<-[$$198, $$onek2] <- test.onek2 condition (and(ge($$onek2.getField(2), 1), le($$onek2.getField(2), 100))) limit 4 [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|
+                              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  |PARTITIONED|
             union ($$355, $$364, $$188) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
             -- UNION_ALL  |PARTITIONED|
               exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
               -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                project ([$$355]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$355] <- [{"two": $$195}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ASSIGN  |PARTITIONED|
-                    limit 4 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_LIMIT  |PARTITIONED|
-                      project ([$$195]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                assign [$$355] <- [{"two": $$195}] project: [$$355] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  limit 4 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_LIMIT  |PARTITIONED|
+                    assign [$$195] <- [$$onek1.getField(2)] project: [$$195] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ASSIGN  |PARTITIONED|
+                      project ([$$onek1]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$195] <- [$$onek1.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ASSIGN  |PARTITIONED|
-                          project ([$$onek1]) [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 []<-[$$199, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), 1), le($$onek1.getField(2), 1000))) limit 4 [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|
-                              data-scan []<-[$$199, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), 1), le($$onek1.getField(2), 1000))) limit 4 [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|
+                              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_PARTITION_EXCHANGE  |PARTITIONED|
-                project ([$$364]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$364] <- [{"two": $$196}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ASSIGN  |PARTITIONED|
-                    limit 4 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_LIMIT  |PARTITIONED|
-                      project ([$$196]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                assign [$$364] <- [{"two": $$196}] project: [$$364] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  limit 4 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_LIMIT  |PARTITIONED|
+                    assign [$$196] <- [$$onek2.getField(2)] project: [$$196] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ASSIGN  |PARTITIONED|
+                      project ([$$onek2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$196] <- [$$onek2.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ASSIGN  |PARTITIONED|
-                          project ([$$onek2]) [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 []<-[$$200, $$onek2] <- test.onek2 condition (and(ge($$onek2.getField(2), 1), le($$onek2.getField(2), 10000))) limit 4 [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|
-                              data-scan []<-[$$200, $$onek2] <- test.onek2 condition (and(ge($$onek2.getField(2), 1), le($$onek2.getField(2), 10000))) limit 4 [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|
+                              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/union/union_opt_1/union_opt_1.9.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.9.plan
index 6208f9e..34e4404 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.9.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.9.plan
@@ -12,43 +12,35 @@
           -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
             limit 4 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
             -- STREAM_LIMIT  |PARTITIONED|
-              project ([$$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                assign [$$52] <- [{"two": $$107}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              assign [$$52] <- [{"two": $$107}] project: [$$52] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ASSIGN  |PARTITIONED|
+                assign [$$107] <- [$$onek1.getField(2)] project: [$$107] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                 -- ASSIGN  |PARTITIONED|
-                  project ([$$107]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  project ([$$onek1]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$107] <- [$$onek1.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
-                      project ([$$onek1]) [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 []<-[$$65, $$onek1] <- test.onek1 condition (gt($$onek1.getField(2), 0)) limit 4 [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|
-                          data-scan []<-[$$65, $$onek1] <- test.onek1 condition (gt($$onek1.getField(2), 0)) limit 4 [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|
+                          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_PARTITION_EXCHANGE  |PARTITIONED|
             limit 4 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
             -- STREAM_LIMIT  |PARTITIONED|
-              project ([$$58]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                assign [$$58] <- [{"two": $$109}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              assign [$$58] <- [{"two": $$109}] project: [$$58] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ASSIGN  |PARTITIONED|
+                assign [$$109] <- [$$onek2.getField(2)] project: [$$109] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                 -- ASSIGN  |PARTITIONED|
-                  project ([$$109]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  project ([$$onek2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$109] <- [$$onek2.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
-                      project ([$$onek2]) [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 []<-[$$66, $$onek2] <- test.onek2 condition (gt($$onek2.getField(2), 0)) limit 4 [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|
-                          data-scan []<-[$$66, $$onek2] <- test.onek2 condition (gt($$onek2.getField(2), 0)) limit 4 [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|
+                          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/union/union_type_cast/union_type_cast.4.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_type_cast/union_type_cast.4.plan
index d0aa765..1f8d6a4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_type_cast/union_type_cast.4.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_type_cast/union_type_cast.4.plan
@@ -18,55 +18,47 @@
                 -- UNION_ALL  |PARTITIONED|
                   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$139, $$68]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$139] <- [cast({"id": $$68, "no_in_response_to": object-remove($$s, "in-response-to")})] [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]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- REPLICATE  |PARTITIONED|
-                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              data-scan []<-[$$68, $$s] <- TinySocial.FacebookMessages [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|
+                    assign [$$139] <- [cast({"id": $$68, "no_in_response_to": object-remove($$s, "in-response-to")})] project: [$$139, $$68] [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]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- REPLICATE  |PARTITIONED|
+                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            data-scan []<-[$$68, $$s] <- TinySocial.FacebookMessages [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  |PARTITIONED|
-                    project ([$$141, $$69]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$141] <- [cast({"id": $$69, "user": $$t})] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
+                    assign [$$141] <- [cast({"id": $$69, "user": $$t})] project: [$$141, $$69] [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]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        data-scan []<-[$$69, $$t] <- TinySocial.FacebookUsers [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  |PARTITIONED|
+                assign [$$140] <- [cast({"id": $$70, "no_author_id": object-remove($$s, "author-id")})] project: [$$140, $$70] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  assign [$$70, $$s] <- [$$68, $$s] project: [$$70, $$s] [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]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- REPLICATE  |PARTITIONED|
                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$69, $$t] <- TinySocial.FacebookUsers [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          data-scan []<-[$$68, $$s] <- TinySocial.FacebookMessages [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  |PARTITIONED|
-                project ([$$140, $$70]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$140] <- [cast({"id": $$70, "no_author_id": object-remove($$s, "author-id")})] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ASSIGN  |PARTITIONED|
-                    project ([$$70, $$s]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$70, $$s] <- [$$68, $$s] [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]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- REPLICATE  |PARTITIONED|
-                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              data-scan []<-[$$68, $$s] <- TinySocial.FacebookMessages [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/view/view-pushdown/view-pushdown.04.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.04.plan
index 235b0e0..a5b3307 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.04.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.04.plan
@@ -2,33 +2,29 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$94]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$94] <- [{"id": $$109, "review": $$114}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-          order (ASC, $$109) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$109(ASC)]  |UNPARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-              limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_LIMIT  |UNPARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                  project ([$$114, $$109]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$109] <- [int64-default-null($$d.getField("id"))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$94] <- [{"id": $$109, "review": $$114}] project: [$$94] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+        order (ASC, $$109) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$109(ASC)]  |UNPARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+            limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_LIMIT  |UNPARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                assign [$$109] <- [int64-default-null($$d.getField("id"))] project: [$$114, $$109] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_LIMIT  |PARTITIONED|
+                    assign [$$114] <- [string-default-null($$d.getField("review"))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                     -- ASSIGN  |PARTITIONED|
-                      limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_LIMIT  |PARTITIONED|
-                        assign [$$114] <- [string-default-null($$d.getField("review"))] [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]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        data-scan []<-[$$d] <- test.ExternalDataset condition (and(not(is-unknown(int64-default-null($$d.getField("year")))), not(is-unknown(int64-default-null($$d.getField("quarter")))), eq(string-default-null($$d.getField("review")), "good"))) limit 3 [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|
-                            data-scan []<-[$$d] <- test.ExternalDataset condition (and(not(is-unknown(int64-default-null($$d.getField("year")))), not(is-unknown(int64-default-null($$d.getField("quarter")))), eq(string-default-null($$d.getField("review")), "good"))) limit 3 [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|
+                            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.06.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.06.plan
index 1d3ef0b..98cefbf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.06.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.06.plan
@@ -6,17 +6,15 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        project ([$$69]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          assign [$$69] <- [{"id": int64-default-null($$d.getField("id")), "review": string-default-null($$d.getField("review"))}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- ASSIGN  |PARTITIONED|
-            limit 3 [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]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                data-scan []<-[$$d] <- test.ExternalDataset limit 3 [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|
+        assign [$$69] <- [{"id": int64-default-null($$d.getField("id")), "review": string-default-null($$d.getField("review"))}] project: [$$69] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- ASSIGN  |PARTITIONED|
+          limit 3 [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]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$d] <- test.ExternalDataset limit 3 [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/view/view-pushdown/view-pushdown.08.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.08.plan
index 4ff57dc3..a98cd9b 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
@@ -2,31 +2,27 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$88]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$88] <- [{"id": $$91, "review": $$95}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-          order (ASC, $$91) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$91(ASC)]  |UNPARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-              limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_LIMIT  |UNPARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                  limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_LIMIT  |PARTITIONED|
-                    project ([$$91, $$95]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$95] <- [$$d.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
+    assign [$$88] <- [{"id": $$91, "review": $$95}] project: [$$88] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+        order (ASC, $$91) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$91(ASC)]  |UNPARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+            limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_LIMIT  |UNPARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STREAM_LIMIT  |PARTITIONED|
+                  assign [$$95] <- [$$d.getField(1)] project: [$$91, $$95] [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]
+                    -- 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: 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|
-                          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: 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|
+                          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 81cc536..e19e209 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
@@ -4,29 +4,25 @@
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$63]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$63] <- [{"id": $$65, "review": $$69}] [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 [$$65(ASC) ]  |PARTITIONED|
-            limit 3 [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]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (topK: 3) (ASC, $$65) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STABLE_SORT [topK: 3] [$$65(ASC)]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$65, $$69]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$69] <- [$$d.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
+      assign [$$63] <- [{"id": $$65, "review": $$69}] project: [$$63] [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 [$$65(ASC) ]  |PARTITIONED|
+          limit 3 [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]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              order (topK: 3) (ASC, $$65) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STABLE_SORT [topK: 3] [$$65(ASC)]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  assign [$$69] <- [$$d.getField(1)] project: [$$65, $$69] [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]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$65, $$d] <- test.DatasetWithKnownField [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|
-                          data-scan []<-[$$65, $$d] <- test.DatasetWithKnownField [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|
+                          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 be33b08..a5c48a4 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
@@ -2,31 +2,27 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$94]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$94] <- [{"id": $$97, "review": $$101}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-          order (ASC, $$97) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$97(ASC)]  |UNPARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-              limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_LIMIT  |UNPARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                  limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_LIMIT  |PARTITIONED|
-                    project ([$$97, $$101]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$101] <- [$$d.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
+    assign [$$94] <- [{"id": $$97, "review": $$101}] project: [$$94] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+        order (ASC, $$97) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$97(ASC)]  |UNPARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+            limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_LIMIT  |UNPARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STREAM_LIMIT  |PARTITIONED|
+                  assign [$$101] <- [$$d.getField(1)] project: [$$97, $$101] [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]
+                    -- 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: 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|
-                          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: 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|
+                          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 f4d1f18..72c87e4 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
@@ -4,29 +4,25 @@
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$69]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$69] <- [{"id": $$71, "review": $$75}] [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 [$$71(ASC) ]  |PARTITIONED|
-            limit 3 [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]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (topK: 3) (ASC, $$71) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STABLE_SORT [topK: 3] [$$71(ASC)]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$71, $$75]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$75] <- [$$d.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
+      assign [$$69] <- [{"id": $$71, "review": $$75}] project: [$$69] [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 [$$71(ASC) ]  |PARTITIONED|
+          limit 3 [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]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              order (topK: 3) (ASC, $$71) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STABLE_SORT [topK: 3] [$$71(ASC)]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  assign [$$75] <- [$$d.getField(1)] project: [$$71, $$75] [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]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$71, $$d] <- test.DatasetWithKnownField [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|
-                          data-scan []<-[$$71, $$d] <- test.DatasetWithKnownField [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|
+                          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_cbo/aggregate-sql/count_dataset/count_dataset.1.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/aggregate-sql/count_dataset/count_dataset.1.plan
index f15e0d9..d0efadf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/aggregate-sql/count_dataset/count_dataset.1.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/aggregate-sql/count_dataset/count_dataset.1.plan
@@ -12,19 +12,15 @@
           -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$27]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select (and(ge($$25, 1), le($$25, 10))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  project ([$$27, $$25]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$25] <- [$$Tweet.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
+              select (and(ge($$25, 1), le($$25, 10))) project: [$$27] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                assign [$$25] <- [$$Tweet.getField(1)] project: [$$27, $$25] [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]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$27, $$Tweet] <- Twitter.Tweet [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|
-                        data-scan []<-[$$27, $$Tweet] <- Twitter.Tweet [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|
+                        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_cbo/aggregate/count_dataset/count_dataset.1.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/aggregate/count_dataset/count_dataset.1.plan
index b3eb407..cc00b25 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/aggregate/count_dataset/count_dataset.1.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/aggregate/count_dataset/count_dataset.1.plan
@@ -12,19 +12,15 @@
           -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$27]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select (and(ge($$25, 1), le($$25, 10))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  project ([$$27, $$25]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$25] <- [$$Tweet.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
+              select (and(ge($$25, 1), le($$25, 10))) project: [$$27] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                assign [$$25] <- [$$Tweet.getField(1)] project: [$$27, $$25] [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]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$27, $$Tweet] <- Twitter.Tweet [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|
-                        data-scan []<-[$$27, $$Tweet] <- Twitter.Tweet [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|
+                        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_cbo/cardinality-estimation/join-queries/join-queries.3.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/join-queries/join-queries.3.plan
index 04f4f79..9d2d093 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/join-queries/join-queries.3.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/join-queries/join-queries.3.plan
@@ -10,23 +10,21 @@
         -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
           exchange [cardinality: 4785.13, op-cost: 0.0, total-cost: 19523.81]
           -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
-            project ([$$35]) [cardinality: 4785.13, op-cost: 0.0, total-cost: 19523.81]
-            -- STREAM_PROJECT  |PARTITIONED|
-              assign [$$35] <- [{"l_linenumber": $$37, "l_partkey": $$l.getField(1), "o_custkey": $$o.getField(1)}] [cardinality: 4785.13, op-cost: 0.0, total-cost: 19523.81]
-              -- ASSIGN  |PARTITIONED|
-                select (gt($$l.getField(4), 10)) [cardinality: 4785.13, op-cost: 12018.16, total-cost: 19523.81]
-                -- STREAM_SELECT  |PARTITIONED|
-                  project ([$$o, $$37, $$l]) [cardinality: 4779.14, op-cost: 0.0, total-cost: 6005.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    exchange [cardinality: 4779.14, op-cost: 0.0, total-cost: 6005.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      unnest-map [$$36, $$37, $$l] <- index-search("LineItem", 0, "Default", "tpch", "LineItem", true, true, 1, $$38, 1, $$38, true, true, true) [cardinality: 4779.14, op-cost: 6005.0, total-cost: 6005.0]
-                      -- BTREE_SEARCH  |PARTITIONED|
-                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$38, $$o] <- tpch.Orders [cardinality: 1500.0, op-cost: 1500.0, total-cost: 1500.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|
+            assign [$$35] <- [{"l_linenumber": $$37, "l_partkey": $$l.getField(1), "o_custkey": $$o.getField(1)}] project: [$$35] [cardinality: 4785.13, op-cost: 0.0, total-cost: 19523.81]
+            -- ASSIGN  |PARTITIONED|
+              select (gt($$l.getField(4), 10)) [cardinality: 4785.13, op-cost: 12018.16, total-cost: 19523.81]
+              -- STREAM_SELECT  |PARTITIONED|
+                project ([$$o, $$37, $$l]) [cardinality: 4779.14, op-cost: 0.0, total-cost: 6005.0]
+                -- STREAM_PROJECT  |PARTITIONED|
+                  exchange [cardinality: 4779.14, op-cost: 0.0, total-cost: 6005.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    unnest-map [$$36, $$37, $$l] <- index-search("LineItem", 0, "Default", "tpch", "LineItem", true, true, 1, $$38, 1, $$38, true, true, true) [cardinality: 4779.14, op-cost: 6005.0, total-cost: 6005.0]
+                    -- BTREE_SEARCH  |PARTITIONED|
+                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        data-scan []<-[$$38, $$o] <- tpch.Orders [cardinality: 1500.0, op-cost: 1500.0, total-cost: 1500.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_cbo/cardinality-estimation/join-queries/join-queries.4.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/join-queries/join-queries.4.plan
index 4f0d55b..0babdc8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/join-queries/join-queries.4.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/join-queries/join-queries.4.plan
@@ -10,25 +10,21 @@
         -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
           exchange [cardinality: 5882.42, op-cost: 0.0, total-cost: 19523.81]
           -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
-            project ([$$34]) [cardinality: 5882.42, op-cost: 0.0, total-cost: 19523.81]
-            -- STREAM_PROJECT  |PARTITIONED|
-              assign [$$34] <- [{"l_orderkey": $$36, "o_custkey": $$o.getField(1)}] [cardinality: 5882.42, op-cost: 0.0, total-cost: 19523.81]
-              -- ASSIGN  |PARTITIONED|
-                project ([$$o, $$36]) [cardinality: 5882.42, op-cost: 0.0, total-cost: 19523.81]
+            assign [$$34] <- [{"l_orderkey": $$36, "o_custkey": $$o.getField(1)}] project: [$$34] [cardinality: 5882.42, op-cost: 0.0, total-cost: 19523.81]
+            -- ASSIGN  |PARTITIONED|
+              select (gt($$l.getField(1), 5)) project: [$$o, $$36] [cardinality: 5882.42, op-cost: 12018.16, total-cost: 19523.81]
+              -- STREAM_SELECT  |PARTITIONED|
+                project ([$$o, $$36, $$l]) [cardinality: 5875.07, op-cost: 0.0, total-cost: 6005.0]
                 -- STREAM_PROJECT  |PARTITIONED|
-                  select (gt($$l.getField(1), 5)) [cardinality: 5882.42, op-cost: 12018.16, total-cost: 19523.81]
-                  -- STREAM_SELECT  |PARTITIONED|
-                    project ([$$o, $$36, $$l]) [cardinality: 5875.07, op-cost: 0.0, total-cost: 6005.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      exchange [cardinality: 5875.07, op-cost: 0.0, total-cost: 6005.0]
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        unnest-map [$$36, $$37, $$l] <- index-search("LineItem", 0, "Default", "tpch", "LineItem", true, true, 1, $$38, 1, $$38, true, true, true) [cardinality: 5875.07, op-cost: 6005.0, total-cost: 6005.0]
-                        -- BTREE_SEARCH  |PARTITIONED|
+                  exchange [cardinality: 5875.07, op-cost: 0.0, total-cost: 6005.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    unnest-map [$$36, $$37, $$l] <- index-search("LineItem", 0, "Default", "tpch", "LineItem", true, true, 1, $$38, 1, $$38, true, true, true) [cardinality: 5875.07, op-cost: 6005.0, total-cost: 6005.0]
+                    -- BTREE_SEARCH  |PARTITIONED|
+                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        data-scan []<-[$$38, $$o] <- tpch.Orders [cardinality: 1500.0, op-cost: 1500.0, total-cost: 1500.0]
+                        -- DATASOURCE_SCAN  |PARTITIONED|
                           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            data-scan []<-[$$38, $$o] <- tpch.Orders [cardinality: 1500.0, op-cost: 1500.0, total-cost: 1500.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|
+                          -- 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_cbo/cardinality-estimation/join-queries/join-queries.5.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/join-queries/join-queries.5.plan
index c981568..4d11aca 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/join-queries/join-queries.5.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/join-queries/join-queries.5.plan
@@ -1,76 +1,68 @@
-distribute result [$$101] [cardinality: 1004.25, op-cost: 0.0, total-cost: 18055.78]
+distribute result [$$101] [cardinality: 1003.05, op-cost: 0.0, total-cost: 18041.34]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 1004.25, op-cost: 0.0, total-cost: 18055.78]
+  exchange [cardinality: 1003.05, op-cost: 0.0, total-cost: 18041.34]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$101]) [cardinality: 1004.25, op-cost: 0.0, total-cost: 18055.78]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$101] <- [{"$1": $$111}] [cardinality: 1004.25, op-cost: 0.0, total-cost: 18055.78]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$111]) [cardinality: 1004.25, op-cost: 0.0, total-cost: 18055.78]
-        -- STREAM_PROJECT  |PARTITIONED|
-          exchange [cardinality: 1004.25, op-cost: 0.0, total-cost: 18055.78]
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            group by ([$$l_partkey := $$115; $$o_orderstatus := $$116; $$c_nationkey := $$117]) decor ([]) {
-                      aggregate [$$111] <- [sql-sum-serial($$114)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- AGGREGATE  |LOCAL|
-                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                   } [cardinality: 1004.25, op-cost: 6025.39, total-cost: 18055.78]
-            -- EXTERNAL_GROUP_BY[$$115, $$116, $$117]  |PARTITIONED|
-              exchange [cardinality: 1004.25, op-cost: 0.0, total-cost: 12030.39]
-              -- HASH_PARTITION_EXCHANGE [$$115, $$116, $$117]  |PARTITIONED|
-                group by ([$$115 := $$102; $$116 := $$103; $$117 := $$104]) decor ([]) {
-                          aggregate [$$114] <- [sql-count-serial(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       } [cardinality: 1004.25, op-cost: 6025.39, total-cost: 12030.39]
-                -- EXTERNAL_GROUP_BY[$$102, $$103, $$104]  |PARTITIONED|
-                  exchange [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$102, $$103, $$104]) [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
+    assign [$$101] <- [{"$1": $$111}] project: [$$101] [cardinality: 1003.05, op-cost: 0.0, total-cost: 18041.34]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$111]) [cardinality: 1003.05, op-cost: 0.0, total-cost: 18041.34]
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange [cardinality: 1003.05, op-cost: 0.0, total-cost: 18041.34]
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          group by ([$$l_partkey := $$115; $$o_orderstatus := $$116; $$c_nationkey := $$117]) decor ([]) {
+                    aggregate [$$111] <- [sql-sum-serial($$114)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                 } [cardinality: 1003.05, op-cost: 6018.17, total-cost: 18041.34]
+          -- EXTERNAL_GROUP_BY[$$115, $$116, $$117]  |PARTITIONED|
+            exchange [cardinality: 1003.05, op-cost: 0.0, total-cost: 12023.17]
+            -- HASH_PARTITION_EXCHANGE [$$115, $$116, $$117]  |PARTITIONED|
+              group by ([$$115 := $$102; $$116 := $$103; $$117 := $$104]) decor ([]) {
+                        aggregate [$$114] <- [sql-count-serial(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- AGGREGATE  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                     } [cardinality: 1003.05, op-cost: 6018.17, total-cost: 12023.17]
+              -- EXTERNAL_GROUP_BY[$$102, $$103, $$104]  |PARTITIONED|
+                exchange [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  assign [$$102] <- [$$l.getField(1)] project: [$$102, $$103, $$104] [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
+                  -- ASSIGN  |PARTITIONED|
+                    project ([$$103, $$104, $$l]) [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
                     -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$102] <- [$$l.getField(1)] [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
-                      -- ASSIGN  |PARTITIONED|
-                        project ([$$103, $$104, $$l]) [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          exchange [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            unnest-map [$$105, $$106, $$l] <- index-search("LineItem", 0, "Default", "tpch", "LineItem", true, true, 1, $$107, 1, $$107, true, true, true) [cardinality: 6010.65, op-cost: 6005.0, total-cost: 6005.0]
-                            -- BTREE_SEARCH  |PARTITIONED|
+                      exchange [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        unnest-map [$$105, $$106, $$l] <- index-search("LineItem", 0, "Default", "tpch", "LineItem", true, true, 1, $$107, 1, $$107, true, true, true) [cardinality: 6010.65, op-cost: 6005.0, total-cost: 6005.0]
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            project ([$$103, $$107, $$104]) [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]
-                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                project ([$$103, $$107, $$104]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                join (eq($$109, $$108)) [cardinality: 1501.41, op-cost: 2101.41, total-cost: 4351.41]
+                                -- HYBRID_HASH_JOIN [$$109][$$108]  |PARTITIONED|
                                   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    join (eq($$109, $$108)) [cardinality: 1503.21, op-cost: 2101.41, total-cost: 4351.41]
-                                    -- HYBRID_HASH_JOIN [$$109][$$108]  |PARTITIONED|
+                                    assign [$$103, $$109] <- [$$o.getField(2), $$o.getField(1)] project: [$$103, $$107, $$109] [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]
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        project ([$$103, $$107, $$109]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          assign [$$103, $$109] <- [$$o.getField(2), $$o.getField(1)] [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]
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              data-scan []<-[$$107, $$o] <- tpch.Orders [cardinality: 1500.0, op-cost: 1500.0, total-cost: 1500.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|
+                                        data-scan []<-[$$107, $$o] <- tpch.Orders [cardinality: 1500.0, op-cost: 1500.0, total-cost: 1500.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]
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    assign [$$104] <- [$$c.getField(3)] project: [$$104, $$108] [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]
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        project ([$$104, $$108]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          assign [$$104] <- [$$c.getField(3)] [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]
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              data-scan []<-[$$108, $$c] <- tpch.Customer [cardinality: 150.0, op-cost: 150.0, total-cost: 150.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|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        data-scan []<-[$$108, $$c] <- tpch.Customer [cardinality: 150.0, op-cost: 150.0, total-cost: 150.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_cbo/cardinality-estimation/join-queries/join-queries.6.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/join-queries/join-queries.6.plan
index e286ea8..743ec55 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/join-queries/join-queries.6.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/join-queries/join-queries.6.plan
@@ -1,72 +1,66 @@
-distribute result [$$101] [cardinality: 1004.25, op-cost: 0.0, total-cost: 34447.61]
+distribute result [$$101] [cardinality: 1003.05, op-cost: 0.0, total-cost: 34411.56]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 1004.25, op-cost: 0.0, total-cost: 34447.61]
+  exchange [cardinality: 1003.05, op-cost: 0.0, total-cost: 34411.56]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$101]) [cardinality: 1004.25, op-cost: 0.0, total-cost: 34447.61]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$101] <- [{"$1": $$111}] [cardinality: 1004.25, op-cost: 0.0, total-cost: 34447.61]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$111]) [cardinality: 1004.25, op-cost: 0.0, total-cost: 34447.61]
-        -- STREAM_PROJECT  |PARTITIONED|
-          exchange [cardinality: 1004.25, op-cost: 0.0, total-cost: 34447.61]
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            group by ([$$l_linenumber := $$115; $$o_orderstatus := $$116; $$c_nationkey := $$117]) decor ([]) {
-                      aggregate [$$111] <- [sql-sum-serial($$114)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- AGGREGATE  |LOCAL|
-                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                   } [cardinality: 1004.25, op-cost: 6025.39, total-cost: 34447.61]
-            -- EXTERNAL_GROUP_BY[$$115, $$116, $$117]  |PARTITIONED|
-              exchange [cardinality: 1004.25, op-cost: 0.0, total-cost: 28422.22]
-              -- HASH_PARTITION_EXCHANGE [$$115, $$116, $$117]  |PARTITIONED|
-                group by ([$$115 := $$106; $$116 := $$103; $$117 := $$104]) decor ([]) {
-                          aggregate [$$114] <- [sql-count-serial(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       } [cardinality: 1004.25, op-cost: 6025.39, total-cost: 28422.22]
-                -- EXTERNAL_GROUP_BY[$$106, $$103, $$104]  |PARTITIONED|
-                  exchange [cardinality: 6025.39, op-cost: 0.0, total-cost: 22396.83]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$106, $$103, $$104]) [cardinality: 6025.39, op-cost: 0.0, total-cost: 22396.83]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      exchange [cardinality: 6025.39, op-cost: 0.0, total-cost: 22396.83]
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        unnest-map [$$105, $$106, $$l] <- index-search("LineItem", 0, "Default", "tpch", "LineItem", true, true, 1, $$107, 1, $$107, true, true, true) [cardinality: 6025.39, op-cost: 12032.57, total-cost: 22396.83]
-                        -- BTREE_SEARCH  |PARTITIONED|
-                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            project ([$$103, $$107, $$104]) [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|
-                                join (eq($$109, $$108)) [cardinality: 1503.21, op-cost: 2101.41, total-cost: 4351.41]
-                                -- HYBRID_HASH_JOIN [$$109][$$108]  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    project ([$$103, $$107, $$109]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      assign [$$103, $$109] <- [$$o.getField(2), $$o.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- ASSIGN  |PARTITIONED|
+    assign [$$101] <- [{"$1": $$111}] project: [$$101] [cardinality: 1003.05, op-cost: 0.0, total-cost: 34411.56]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$111]) [cardinality: 1003.05, op-cost: 0.0, total-cost: 34411.56]
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange [cardinality: 1003.05, op-cost: 0.0, total-cost: 34411.56]
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          group by ([$$l_linenumber := $$115; $$o_orderstatus := $$116; $$c_nationkey := $$117]) decor ([]) {
+                    aggregate [$$111] <- [sql-sum-serial($$114)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                 } [cardinality: 1003.05, op-cost: 6018.17, total-cost: 34411.56]
+          -- EXTERNAL_GROUP_BY[$$115, $$116, $$117]  |PARTITIONED|
+            exchange [cardinality: 1003.05, op-cost: 0.0, total-cost: 28393.39]
+            -- HASH_PARTITION_EXCHANGE [$$115, $$116, $$117]  |PARTITIONED|
+              group by ([$$115 := $$106; $$116 := $$103; $$117 := $$104]) decor ([]) {
+                        aggregate [$$114] <- [sql-count-serial(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- AGGREGATE  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                     } [cardinality: 1003.05, op-cost: 6018.17, total-cost: 28393.39]
+              -- EXTERNAL_GROUP_BY[$$106, $$103, $$104]  |PARTITIONED|
+                exchange [cardinality: 6018.17, op-cost: 0.0, total-cost: 22375.22]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  project ([$$106, $$103, $$104]) [cardinality: 6018.17, op-cost: 0.0, total-cost: 22375.22]
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    exchange [cardinality: 6018.17, op-cost: 0.0, total-cost: 22375.22]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      unnest-map [$$105, $$106, $$l] <- index-search("LineItem", 0, "Default", "tpch", "LineItem", true, true, 1, $$107, 1, $$107, true, true, true) [cardinality: 6018.17, op-cost: 12018.16, total-cost: 22375.22]
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          project ([$$103, $$107, $$104]) [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|
+                              join (eq($$109, $$108)) [cardinality: 1501.41, op-cost: 2101.41, total-cost: 4351.41]
+                              -- HYBRID_HASH_JOIN [$$109][$$108]  |PARTITIONED|
+                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  assign [$$103, $$109] <- [$$o.getField(2), $$o.getField(1)] project: [$$103, $$107, $$109] [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]
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      data-scan []<-[$$107, $$o] <- tpch.Orders [cardinality: 1500.0, op-cost: 1500.0, total-cost: 1500.0]
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
                                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          data-scan []<-[$$107, $$o] <- tpch.Orders [cardinality: 1500.0, op-cost: 1500.0, total-cost: 1500.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]
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                    project ([$$104, $$108]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      assign [$$104] <- [$$c.getField(3)] [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|
+                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  assign [$$104] <- [$$c.getField(3)] project: [$$104, $$108] [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]
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      data-scan []<-[$$108, $$c] <- tpch.Customer [cardinality: 150.0, op-cost: 150.0, total-cost: 150.0]
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
                                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          data-scan []<-[$$108, $$c] <- tpch.Customer [cardinality: 150.0, op-cost: 150.0, total-cost: 150.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|
+                                          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_cbo/cardinality-estimation/join-queries/join-queries.7.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/join-queries/join-queries.7.plan
index ccdd8c9..9e4eff7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/join-queries/join-queries.7.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/join-queries/join-queries.7.plan
@@ -1,72 +1,66 @@
-distribute result [$$101] [cardinality: 1004.25, op-cost: 0.0, total-cost: 34447.61]
+distribute result [$$101] [cardinality: 1003.05, op-cost: 0.0, total-cost: 34411.56]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 1004.25, op-cost: 0.0, total-cost: 34447.61]
+  exchange [cardinality: 1003.05, op-cost: 0.0, total-cost: 34411.56]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$101]) [cardinality: 1004.25, op-cost: 0.0, total-cost: 34447.61]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$101] <- [{"$1": $$111}] [cardinality: 1004.25, op-cost: 0.0, total-cost: 34447.61]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$111]) [cardinality: 1004.25, op-cost: 0.0, total-cost: 34447.61]
-        -- STREAM_PROJECT  |PARTITIONED|
-          exchange [cardinality: 1004.25, op-cost: 0.0, total-cost: 34447.61]
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            group by ([$$l_linenumber := $$115; $$c_nationkey := $$116; $$o_orderstatus := $$117]) decor ([]) {
-                      aggregate [$$111] <- [sql-sum-serial($$114)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- AGGREGATE  |LOCAL|
-                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                   } [cardinality: 1004.25, op-cost: 6025.39, total-cost: 34447.61]
-            -- EXTERNAL_GROUP_BY[$$115, $$116, $$117]  |PARTITIONED|
-              exchange [cardinality: 1004.25, op-cost: 0.0, total-cost: 28422.22]
-              -- HASH_PARTITION_EXCHANGE [$$115, $$116, $$117]  |PARTITIONED|
-                group by ([$$115 := $$106; $$116 := $$103; $$117 := $$104]) decor ([]) {
-                          aggregate [$$114] <- [sql-count-serial(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       } [cardinality: 1004.25, op-cost: 6025.39, total-cost: 28422.22]
-                -- EXTERNAL_GROUP_BY[$$106, $$103, $$104]  |PARTITIONED|
-                  exchange [cardinality: 6025.39, op-cost: 0.0, total-cost: 22396.83]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$106, $$103, $$104]) [cardinality: 6025.39, op-cost: 0.0, total-cost: 22396.83]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      exchange [cardinality: 6025.39, op-cost: 0.0, total-cost: 22396.83]
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        unnest-map [$$105, $$106, $$l] <- index-search("LineItem", 0, "Default", "tpch", "LineItem", true, true, 1, $$107, 1, $$107, true, true, true) [cardinality: 6025.39, op-cost: 12032.57, total-cost: 22396.83]
-                        -- BTREE_SEARCH  |PARTITIONED|
-                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            project ([$$104, $$107, $$103]) [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|
-                                join (eq($$109, $$108)) [cardinality: 1503.21, op-cost: 2101.41, total-cost: 4351.41]
-                                -- HYBRID_HASH_JOIN [$$109][$$108]  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    project ([$$104, $$107, $$109]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      assign [$$104, $$109] <- [$$o.getField(2), $$o.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- ASSIGN  |PARTITIONED|
+    assign [$$101] <- [{"$1": $$111}] project: [$$101] [cardinality: 1003.05, op-cost: 0.0, total-cost: 34411.56]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$111]) [cardinality: 1003.05, op-cost: 0.0, total-cost: 34411.56]
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange [cardinality: 1003.05, op-cost: 0.0, total-cost: 34411.56]
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          group by ([$$l_linenumber := $$115; $$c_nationkey := $$116; $$o_orderstatus := $$117]) decor ([]) {
+                    aggregate [$$111] <- [sql-sum-serial($$114)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                 } [cardinality: 1003.05, op-cost: 6018.17, total-cost: 34411.56]
+          -- EXTERNAL_GROUP_BY[$$115, $$116, $$117]  |PARTITIONED|
+            exchange [cardinality: 1003.05, op-cost: 0.0, total-cost: 28393.39]
+            -- HASH_PARTITION_EXCHANGE [$$115, $$116, $$117]  |PARTITIONED|
+              group by ([$$115 := $$106; $$116 := $$103; $$117 := $$104]) decor ([]) {
+                        aggregate [$$114] <- [sql-count-serial(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- AGGREGATE  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                     } [cardinality: 1003.05, op-cost: 6018.17, total-cost: 28393.39]
+              -- EXTERNAL_GROUP_BY[$$106, $$103, $$104]  |PARTITIONED|
+                exchange [cardinality: 6018.17, op-cost: 0.0, total-cost: 22375.22]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  project ([$$106, $$103, $$104]) [cardinality: 6018.17, op-cost: 0.0, total-cost: 22375.22]
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    exchange [cardinality: 6018.17, op-cost: 0.0, total-cost: 22375.22]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      unnest-map [$$105, $$106, $$l] <- index-search("LineItem", 0, "Default", "tpch", "LineItem", true, true, 1, $$107, 1, $$107, true, true, true) [cardinality: 6018.17, op-cost: 12018.16, total-cost: 22375.22]
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          project ([$$104, $$107, $$103]) [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|
+                              join (eq($$109, $$108)) [cardinality: 1501.41, op-cost: 2101.41, total-cost: 4351.41]
+                              -- HYBRID_HASH_JOIN [$$109][$$108]  |PARTITIONED|
+                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  assign [$$104, $$109] <- [$$o.getField(2), $$o.getField(1)] project: [$$104, $$107, $$109] [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]
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      data-scan []<-[$$107, $$o] <- tpch.Orders [cardinality: 1500.0, op-cost: 1500.0, total-cost: 1500.0]
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
                                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          data-scan []<-[$$107, $$o] <- tpch.Orders [cardinality: 1500.0, op-cost: 1500.0, total-cost: 1500.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]
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                    project ([$$103, $$108]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      assign [$$103] <- [$$c.getField(3)] [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|
+                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  assign [$$103] <- [$$c.getField(3)] project: [$$103, $$108] [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]
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      data-scan []<-[$$108, $$c] <- tpch.Customer [cardinality: 150.0, op-cost: 150.0, total-cost: 150.0]
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
                                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          data-scan []<-[$$108, $$c] <- tpch.Customer [cardinality: 150.0, op-cost: 150.0, total-cost: 150.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|
+                                          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_cbo/cardinality-estimation/join-queries/join-queries.8.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/join-queries/join-queries.8.plan
index 5e89200..baa6c88 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/join-queries/join-queries.8.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/join-queries/join-queries.8.plan
@@ -1,122 +1,108 @@
-distribute result [$$119] [cardinality: 25.0, op-cost: 0.0, total-cost: 7223.81]
+distribute result [$$119] [cardinality: 25.0, op-cost: 0.0, total-cost: 7109.8]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 25.0, op-cost: 0.0, total-cost: 7223.81]
+  exchange [cardinality: 25.0, op-cost: 0.0, total-cost: 7109.8]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$119]) [cardinality: 25.0, op-cost: 0.0, total-cost: 7223.81]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$119] <- [{"n_name": $$n_name, "revenue": $$132}] [cardinality: 25.0, op-cost: 0.0, total-cost: 7223.81]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 25.0, op-cost: 0.0, total-cost: 7223.81]
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          group by ([$$n_name := $$142]) decor ([]) {
-                    aggregate [$$132] <- [global-sql-sum-serial($$141)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 25.0, op-cost: 110.34, total-cost: 7223.81]
-          -- EXTERNAL_GROUP_BY[$$142]  |PARTITIONED|
-            exchange [cardinality: 25.0, op-cost: 0.0, total-cost: 7113.47]
-            -- HASH_PARTITION_EXCHANGE [$$142]  |PARTITIONED|
-              group by ([$$142 := $$120]) decor ([]) {
-                        aggregate [$$141] <- [local-sql-sum-serial(numeric-multiply($$139, numeric-subtract(1, $$140)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 25.0, op-cost: 110.34, total-cost: 7113.47]
-              -- EXTERNAL_GROUP_BY[$$120]  |PARTITIONED|
-                exchange [cardinality: 110.34, op-cost: 0.0, total-cost: 7003.13]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$139, $$140, $$120]) [cardinality: 110.34, op-cost: 0.0, total-cost: 7003.13]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    exchange [cardinality: 110.34, op-cost: 0.0, total-cost: 7003.13]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      join (and(eq($$130, $$128), eq($$133, $$127))) [cardinality: 110.34, op-cost: 1062.24, total-cost: 7003.13]
-                      -- HYBRID_HASH_JOIN [$$128, $$133][$$130, $$127]  |PARTITIONED|
-                        exchange [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          project ([$$139, $$140, $$120, $$128, $$133]) [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
+    assign [$$119] <- [{"n_name": $$n_name, "revenue": $$132}] project: [$$119] [cardinality: 25.0, op-cost: 0.0, total-cost: 7109.8]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 25.0, op-cost: 0.0, total-cost: 7109.8]
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        group by ([$$n_name := $$142]) decor ([]) {
+                  aggregate [$$132] <- [global-sql-sum-serial($$141)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 25.0, op-cost: 110.61, total-cost: 7109.8]
+        -- EXTERNAL_GROUP_BY[$$142]  |PARTITIONED|
+          exchange [cardinality: 25.0, op-cost: 0.0, total-cost: 6999.19]
+          -- HASH_PARTITION_EXCHANGE [$$142]  |PARTITIONED|
+            group by ([$$142 := $$120]) decor ([]) {
+                      aggregate [$$141] <- [local-sql-sum-serial(numeric-multiply($$139, numeric-subtract(1, $$140)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 25.0, op-cost: 110.61, total-cost: 6999.19]
+            -- EXTERNAL_GROUP_BY[$$120]  |PARTITIONED|
+              exchange [cardinality: 110.61, op-cost: 0.0, total-cost: 6888.58]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                project ([$$139, $$140, $$120]) [cardinality: 110.61, op-cost: 0.0, total-cost: 6888.58]
+                -- STREAM_PROJECT  |PARTITIONED|
+                  exchange [cardinality: 110.61, op-cost: 0.0, total-cost: 6888.58]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    join (and(eq($$130, $$128), eq($$133, $$127))) [cardinality: 110.61, op-cost: 1035.49, total-cost: 6888.58]
+                    -- HYBRID_HASH_JOIN [$$128, $$133][$$130, $$127]  |PARTITIONED|
+                      exchange [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        assign [$$140, $$139, $$133] <- [$$l.getField(6), $$l.getField(5), $$l.getField(2)] project: [$$139, $$140, $$120, $$128, $$133] [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
+                        -- ASSIGN  |PARTITIONED|
+                          project ([$$120, $$128, $$l]) [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
                           -- STREAM_PROJECT  |PARTITIONED|
-                            assign [$$140, $$139, $$133] <- [$$l.getField(6), $$l.getField(5), $$l.getField(2)] [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
-                            -- ASSIGN  |PARTITIONED|
-                              project ([$$120, $$128, $$l]) [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                exchange [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  unnest-map [$$125, $$126, $$l] <- index-search("LineItem", 0, "Default", "tpch", "LineItem", true, true, 1, $$124, 1, $$124, true, true, true) [cardinality: 6010.65, op-cost: 6005.0, total-cost: 6005.0]
-                                  -- BTREE_SEARCH  |PARTITIONED|
+                            exchange [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              unnest-map [$$125, $$126, $$l] <- index-search("LineItem", 0, "Default", "tpch", "LineItem", true, true, 1, $$124, 1, $$124, true, true, true) [cardinality: 6010.65, op-cost: 6005.0, total-cost: 6005.0]
+                              -- BTREE_SEARCH  |PARTITIONED|
+                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  project ([$$120, $$128, $$124]) [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]
-                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                      project ([$$120, $$128, $$124]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      join (eq($$123, $$136)) [cardinality: 248.35, op-cost: 398.35, total-cost: 2821.71]
+                                      -- HYBRID_HASH_JOIN [$$123][$$136]  |PARTITIONED|
                                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          join (eq($$123, $$136)) [cardinality: 255.03, op-cost: 402.2, total-cost: 2829.4]
-                                          -- HYBRID_HASH_JOIN [$$123][$$136]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$123]  |PARTITIONED|
+                                          project ([$$120, $$128, $$123]) [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]
-                                            -- HASH_PARTITION_EXCHANGE [$$123]  |PARTITIONED|
-                                              project ([$$120, $$128, $$123]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              join (eq($$131, $$128)) [cardinality: 150.0, op-cost: 175.0, total-cost: 525.0]
+                                              -- HYBRID_HASH_JOIN [$$131][$$128]  |PARTITIONED|
                                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  join (eq($$131, $$128)) [cardinality: 153.85, op-cost: 175.0, total-cost: 525.0]
-                                                  -- HYBRID_HASH_JOIN [$$131][$$128]  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$131]  |PARTITIONED|
+                                                  assign [$$131] <- [$$c.getField(3)] project: [$$123, $$131] [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]
-                                                    -- HASH_PARTITION_EXCHANGE [$$131]  |PARTITIONED|
-                                                      project ([$$123, $$131]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        assign [$$131] <- [$$c.getField(3)] [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]
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            data-scan []<-[$$123, $$c] <- tpch.Customer [cardinality: 150.0, op-cost: 150.0, total-cost: 150.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|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      data-scan []<-[$$123, $$c] <- tpch.Customer [cardinality: 150.0, op-cost: 150.0, total-cost: 150.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 [$$128]  |PARTITIONED|
+                                                  assign [$$120] <- [$$n.getField(1)] project: [$$120, $$128] [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]
-                                                    -- HASH_PARTITION_EXCHANGE [$$128]  |PARTITIONED|
-                                                      project ([$$120, $$128]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        assign [$$120] <- [$$n.getField(1)] [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]
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            data-scan []<-[$$128, $$n] <- tpch.Nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.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 [$$136]  |PARTITIONED|
-                                              project ([$$124, $$136]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                select (and(lt($$121, "1994-01-01"), ge($$121, "1993-01-01"))) [cardinality: 248.35, op-cost: 0.0, total-cost: 1500.0]
-                                                -- STREAM_SELECT  |PARTITIONED|
-                                                  project ([$$124, $$136, $$121]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    assign [$$136, $$121] <- [$$o.getField(1), $$o.getField(4)] [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]
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        data-scan []<-[$$124, $$o] <- tpch.Orders [cardinality: 1500.0, op-cost: 1500.0, total-cost: 1500.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: 10.0, op-cost: 40.0, total-cost: 50.0]
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                          project ([$$130, $$127]) [cardinality: 10.0, op-cost: 0.0, total-cost: 10.0]
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            assign [$$130] <- [$$s.getField(3)] [cardinality: 10.0, op-cost: 0.0, total-cost: 10.0]
-                            -- ASSIGN  |PARTITIONED|
-                              exchange [cardinality: 10.0, op-cost: 40.0, total-cost: 50.0]
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      data-scan []<-[$$128, $$n] <- tpch.Nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.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 [$$136]  |PARTITIONED|
+                                          select (and(lt($$121, "1994-01-01"), ge($$121, "1993-01-01"))) project: [$$124, $$136] [cardinality: 248.35, op-cost: 0.0, total-cost: 1500.0]
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            assign [$$136, $$121] <- [$$o.getField(1), $$o.getField(4)] project: [$$124, $$136, $$121] [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]
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                data-scan []<-[$$124, $$o] <- tpch.Orders [cardinality: 1500.0, op-cost: 1500.0, total-cost: 1500.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: 10.0, op-cost: 40.0, total-cost: 50.0]
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        assign [$$130] <- [$$s.getField(3)] project: [$$130, $$127] [cardinality: 10.0, op-cost: 0.0, total-cost: 10.0]
+                        -- ASSIGN  |PARTITIONED|
+                          exchange [cardinality: 10.0, op-cost: 40.0, total-cost: 50.0]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            data-scan []<-[$$127, $$s] <- tpch.Supplier [cardinality: 10.0, op-cost: 10.0, total-cost: 10.0]
+                            -- DATASOURCE_SCAN  |PARTITIONED|
+                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                data-scan []<-[$$127, $$s] <- tpch.Supplier [cardinality: 10.0, op-cost: 10.0, total-cost: 10.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|
+                                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_cbo/cardinality-estimation/single-collection-queries/single-collection-queries.3.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/single-collection-queries/single-collection-queries.3.plan
index 8a6990c..18b16fd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/single-collection-queries/single-collection-queries.3.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/single-collection-queries/single-collection-queries.3.plan
@@ -2,45 +2,41 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 100.0, op-cost: 0.0, total-cost: 4495.77]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$48]) [cardinality: 100.0, op-cost: 0.0, total-cost: 4495.77]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$48] <- [{"$1": $$51}] [cardinality: 100.0, op-cost: 0.0, total-cost: 4495.77]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$51]) [cardinality: 100.0, op-cost: 0.0, total-cost: 4495.77]
-        -- STREAM_PROJECT  |PARTITIONED|
-          exchange [cardinality: 100.0, op-cost: 0.0, total-cost: 4495.77]
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            group by ([$$o_custkey := $$54]) decor ([]) {
-                      aggregate [$$51] <- [sql-sum-serial($$53)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- AGGREGATE  |LOCAL|
-                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                   } [cardinality: 100.0, op-cost: 1498.59, total-cost: 4495.77]
-            -- EXTERNAL_GROUP_BY[$$54]  |PARTITIONED|
-              exchange [cardinality: 100.0, op-cost: 0.0, total-cost: 2997.18]
-              -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
-                group by ([$$54 := $$49]) decor ([]) {
-                          aggregate [$$53] <- [sql-count-serial(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       } [cardinality: 100.0, op-cost: 1498.59, total-cost: 2997.18]
-                -- EXTERNAL_GROUP_BY[$$49]  |PARTITIONED|
-                  exchange [cardinality: 1498.59, op-cost: 0.0, total-cost: 1498.59]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$49]) [cardinality: 1498.59, op-cost: 0.0, total-cost: 1498.59]
+    assign [$$48] <- [{"$1": $$51}] project: [$$48] [cardinality: 100.0, op-cost: 0.0, total-cost: 4495.77]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$51]) [cardinality: 100.0, op-cost: 0.0, total-cost: 4495.77]
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange [cardinality: 100.0, op-cost: 0.0, total-cost: 4495.77]
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          group by ([$$o_custkey := $$54]) decor ([]) {
+                    aggregate [$$51] <- [sql-sum-serial($$53)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                 } [cardinality: 100.0, op-cost: 1498.59, total-cost: 4495.77]
+          -- EXTERNAL_GROUP_BY[$$54]  |PARTITIONED|
+            exchange [cardinality: 100.0, op-cost: 0.0, total-cost: 2997.18]
+            -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+              group by ([$$54 := $$49]) decor ([]) {
+                        aggregate [$$53] <- [sql-count-serial(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- AGGREGATE  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                     } [cardinality: 100.0, op-cost: 1498.59, total-cost: 2997.18]
+              -- EXTERNAL_GROUP_BY[$$49]  |PARTITIONED|
+                exchange [cardinality: 1498.59, op-cost: 0.0, total-cost: 1498.59]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  assign [$$49] <- [$$o.getField(1)] project: [$$49] [cardinality: 1498.59, op-cost: 0.0, total-cost: 1498.59]
+                  -- ASSIGN  |PARTITIONED|
+                    project ([$$o]) [cardinality: 1498.59, op-cost: 0.0, total-cost: 1498.59]
                     -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$49] <- [$$o.getField(1)] [cardinality: 1498.59, op-cost: 0.0, total-cost: 1498.59]
-                      -- ASSIGN  |PARTITIONED|
-                        project ([$$o]) [cardinality: 1498.59, op-cost: 0.0, total-cost: 1498.59]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          exchange [cardinality: 1498.59, op-cost: 0.0, total-cost: 1498.59]
+                      exchange [cardinality: 1498.59, op-cost: 0.0, total-cost: 1498.59]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        unnest-map [$$50, $$o] <- index-search("Orders", 0, "Default", "tpch", "Orders", false, false, 1, $$55, 0, false, true, false) [cardinality: 1498.59, op-cost: 1498.59, total-cost: 1498.59]
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            unnest-map [$$50, $$o] <- index-search("Orders", 0, "Default", "tpch", "Orders", false, false, 1, $$55, 0, false, true, false) [cardinality: 1498.59, op-cost: 1498.59, total-cost: 1498.59]
-                            -- BTREE_SEARCH  |PARTITIONED|
-                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                assign [$$55] <- [2] [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|
+                            assign [$$55] <- [2] [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_cbo/cardinality-estimation/single-collection-queries/single-collection-queries.4.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/single-collection-queries/single-collection-queries.4.plan
index 6ea6385..5c735b9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/single-collection-queries/single-collection-queries.4.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/single-collection-queries/single-collection-queries.4.plan
@@ -2,45 +2,41 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 1495.77, op-cost: 0.0, total-cost: 4487.31]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$48]) [cardinality: 1495.77, op-cost: 0.0, total-cost: 4487.31]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$48] <- [{"$1": $$51}] [cardinality: 1495.77, op-cost: 0.0, total-cost: 4487.31]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$51]) [cardinality: 1495.77, op-cost: 0.0, total-cost: 4487.31]
-        -- STREAM_PROJECT  |PARTITIONED|
-          exchange [cardinality: 1495.77, op-cost: 0.0, total-cost: 4487.31]
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            group by ([$$o_orderdate := $$54]) decor ([]) {
-                      aggregate [$$51] <- [sql-sum-serial($$53)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- AGGREGATE  |LOCAL|
-                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                   } [cardinality: 1495.77, op-cost: 1495.77, total-cost: 4487.31]
-            -- EXTERNAL_GROUP_BY[$$54]  |PARTITIONED|
-              exchange [cardinality: 1495.77, op-cost: 0.0, total-cost: 2991.54]
-              -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
-                group by ([$$54 := $$49]) decor ([]) {
-                          aggregate [$$53] <- [sql-count-serial(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       } [cardinality: 1495.77, op-cost: 1495.77, total-cost: 2991.54]
-                -- EXTERNAL_GROUP_BY[$$49]  |PARTITIONED|
-                  exchange [cardinality: 1495.77, op-cost: 0.0, total-cost: 1495.77]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$49]) [cardinality: 1495.77, op-cost: 0.0, total-cost: 1495.77]
+    assign [$$48] <- [{"$1": $$51}] project: [$$48] [cardinality: 1495.77, op-cost: 0.0, total-cost: 4487.31]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$51]) [cardinality: 1495.77, op-cost: 0.0, total-cost: 4487.31]
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange [cardinality: 1495.77, op-cost: 0.0, total-cost: 4487.31]
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          group by ([$$o_orderdate := $$54]) decor ([]) {
+                    aggregate [$$51] <- [sql-sum-serial($$53)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                 } [cardinality: 1495.77, op-cost: 1495.77, total-cost: 4487.31]
+          -- EXTERNAL_GROUP_BY[$$54]  |PARTITIONED|
+            exchange [cardinality: 1495.77, op-cost: 0.0, total-cost: 2991.54]
+            -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+              group by ([$$54 := $$49]) decor ([]) {
+                        aggregate [$$53] <- [sql-count-serial(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- AGGREGATE  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                     } [cardinality: 1495.77, op-cost: 1495.77, total-cost: 2991.54]
+              -- EXTERNAL_GROUP_BY[$$49]  |PARTITIONED|
+                exchange [cardinality: 1495.77, op-cost: 0.0, total-cost: 1495.77]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  assign [$$49] <- [$$o.getField(4)] project: [$$49] [cardinality: 1495.77, op-cost: 0.0, total-cost: 1495.77]
+                  -- ASSIGN  |PARTITIONED|
+                    project ([$$o]) [cardinality: 1495.77, op-cost: 0.0, total-cost: 1495.77]
                     -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$49] <- [$$o.getField(4)] [cardinality: 1495.77, op-cost: 0.0, total-cost: 1495.77]
-                      -- ASSIGN  |PARTITIONED|
-                        project ([$$o]) [cardinality: 1495.77, op-cost: 0.0, total-cost: 1495.77]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          exchange [cardinality: 1495.77, op-cost: 0.0, total-cost: 1495.77]
+                      exchange [cardinality: 1495.77, op-cost: 0.0, total-cost: 1495.77]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        unnest-map [$$50, $$o] <- index-search("Orders", 0, "Default", "tpch", "Orders", false, false, 1, $$55, 0, false, true, false) [cardinality: 1495.77, op-cost: 1495.77, total-cost: 1495.77]
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            unnest-map [$$50, $$o] <- index-search("Orders", 0, "Default", "tpch", "Orders", false, false, 1, $$55, 0, false, true, false) [cardinality: 1495.77, op-cost: 1495.77, total-cost: 1495.77]
-                            -- BTREE_SEARCH  |PARTITIONED|
-                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                assign [$$55] <- [5] [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|
+                            assign [$$55] <- [5] [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_cbo/cardinality-estimation/single-collection-queries/single-collection-queries.5.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/single-collection-queries/single-collection-queries.5.plan
index ce1fe82..cdcaa28 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/single-collection-queries/single-collection-queries.5.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/single-collection-queries/single-collection-queries.5.plan
@@ -2,43 +2,39 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 1298.0, op-cost: 0.0, total-cost: 15506.8]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$51]) [cardinality: 1298.0, op-cost: 0.0, total-cost: 15506.8]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$51] <- [{"$1": $$55}] [cardinality: 1298.0, op-cost: 0.0, total-cost: 15506.8]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$55]) [cardinality: 1298.0, op-cost: 0.0, total-cost: 15506.8]
-        -- STREAM_PROJECT  |PARTITIONED|
-          exchange [cardinality: 1298.0, op-cost: 0.0, total-cost: 15506.8]
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            group by ([$$l_orderkey := $$59]) decor ([]) {
-                      aggregate [$$55] <- [sql-sum-serial($$58)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- AGGREGATE  |LOCAL|
-                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                   } [cardinality: 1298.0, op-cost: 4750.9, total-cost: 15506.8]
-            -- EXTERNAL_GROUP_BY[$$59]  |PARTITIONED|
-              exchange [cardinality: 1298.0, op-cost: 0.0, total-cost: 10755.9]
-              -- HASH_PARTITION_EXCHANGE [$$59]  |PARTITIONED|
-                group by ([$$59 := $$53]) decor ([]) {
-                          aggregate [$$58] <- [sql-count-serial(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       } [cardinality: 1298.0, op-cost: 4750.9, total-cost: 10755.9]
-                -- EXTERNAL_GROUP_BY[$$53]  |PARTITIONED|
-                  exchange [cardinality: 4750.9, op-cost: 0.0, total-cost: 6005.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$53]) [cardinality: 4750.9, op-cost: 0.0, total-cost: 6005.0]
+    assign [$$51] <- [{"$1": $$55}] project: [$$51] [cardinality: 1298.0, op-cost: 0.0, total-cost: 15506.8]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$55]) [cardinality: 1298.0, op-cost: 0.0, total-cost: 15506.8]
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange [cardinality: 1298.0, op-cost: 0.0, total-cost: 15506.8]
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          group by ([$$l_orderkey := $$59]) decor ([]) {
+                    aggregate [$$55] <- [sql-sum-serial($$58)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                 } [cardinality: 1298.0, op-cost: 4750.9, total-cost: 15506.8]
+          -- EXTERNAL_GROUP_BY[$$59]  |PARTITIONED|
+            exchange [cardinality: 1298.0, op-cost: 0.0, total-cost: 10755.9]
+            -- HASH_PARTITION_EXCHANGE [$$59]  |PARTITIONED|
+              group by ([$$59 := $$53]) decor ([]) {
+                        aggregate [$$58] <- [sql-count-serial(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- AGGREGATE  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                     } [cardinality: 1298.0, op-cost: 4750.9, total-cost: 10755.9]
+              -- EXTERNAL_GROUP_BY[$$53]  |PARTITIONED|
+                exchange [cardinality: 4750.9, op-cost: 0.0, total-cost: 6005.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  select (and(gt($$l.getField(4), 10), gt($$l.getField(1), 1))) project: [$$53] [cardinality: 4750.9, op-cost: 0.0, total-cost: 6005.0]
+                  -- STREAM_SELECT  |PARTITIONED|
+                    project ([$$53, $$l]) [cardinality: 6005.0, op-cost: 0.0, total-cost: 6005.0]
                     -- STREAM_PROJECT  |PARTITIONED|
-                      select (and(gt($$l.getField(4), 10), gt($$l.getField(1), 1))) [cardinality: 4750.9, op-cost: 0.0, total-cost: 6005.0]
-                      -- STREAM_SELECT  |PARTITIONED|
-                        project ([$$53, $$l]) [cardinality: 6005.0, op-cost: 0.0, total-cost: 6005.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          exchange [cardinality: 6005.0, op-cost: 0.0, total-cost: 6005.0]
+                      exchange [cardinality: 6005.0, op-cost: 0.0, total-cost: 6005.0]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        data-scan []<-[$$53, $$54, $$l] <- tpch.LineItem [cardinality: 6005.0, op-cost: 6005.0, total-cost: 6005.0]
+                        -- DATASOURCE_SCAN  |PARTITIONED|
+                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            data-scan []<-[$$53, $$54, $$l] <- tpch.LineItem [cardinality: 6005.0, op-cost: 6005.0, total-cost: 6005.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|
\ No newline at end of file
+                            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_cbo/cardinality-estimation/single-collection-queries/single-collection-queries.6.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/single-collection-queries/single-collection-queries.6.plan
index d6d4f39..61cde6a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/single-collection-queries/single-collection-queries.6.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/single-collection-queries/single-collection-queries.6.plan
@@ -2,41 +2,37 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 1543.0, op-cost: 0.0, total-cost: 13427.92]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$51]) [cardinality: 1543.0, op-cost: 0.0, total-cost: 13427.92]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$51] <- [{"$1": $$55}] [cardinality: 1543.0, op-cost: 0.0, total-cost: 13427.92]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$55]) [cardinality: 1543.0, op-cost: 0.0, total-cost: 13427.92]
-        -- STREAM_PROJECT  |PARTITIONED|
-          exchange [cardinality: 1543.0, op-cost: 0.0, total-cost: 13427.92]
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            group by ([$$l_orderkey := $$59]) decor ([]) {
-                      aggregate [$$55] <- [sql-sum-serial($$58)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- AGGREGATE  |LOCAL|
-                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                   } [cardinality: 1543.0, op-cost: 3711.46, total-cost: 13427.92]
-            -- EXTERNAL_GROUP_BY[$$59]  |PARTITIONED|
-              exchange [cardinality: 1543.0, op-cost: 0.0, total-cost: 9716.46]
-              -- HASH_PARTITION_EXCHANGE [$$59]  |PARTITIONED|
-                group by ([$$59 := $$53]) decor ([]) {
-                          aggregate [$$58] <- [sql-count-serial(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       } [cardinality: 1543.0, op-cost: 3711.46, total-cost: 9716.46]
-                -- EXTERNAL_GROUP_BY[$$53]  |PARTITIONED|
-                  exchange [cardinality: 3711.46, op-cost: 0.0, total-cost: 6005.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$53]) [cardinality: 3711.46, op-cost: 0.0, total-cost: 6005.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      select (and(le($$54, 4), gt($$l.getField(4), 10))) [cardinality: 3711.46, op-cost: 0.0, total-cost: 6005.0]
-                      -- STREAM_SELECT  |PARTITIONED|
-                        exchange [cardinality: 6005.0, op-cost: 0.0, total-cost: 6005.0]
+    assign [$$51] <- [{"$1": $$55}] project: [$$51] [cardinality: 1543.0, op-cost: 0.0, total-cost: 13427.92]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$55]) [cardinality: 1543.0, op-cost: 0.0, total-cost: 13427.92]
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange [cardinality: 1543.0, op-cost: 0.0, total-cost: 13427.92]
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          group by ([$$l_orderkey := $$59]) decor ([]) {
+                    aggregate [$$55] <- [sql-sum-serial($$58)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                 } [cardinality: 1543.0, op-cost: 3711.46, total-cost: 13427.92]
+          -- EXTERNAL_GROUP_BY[$$59]  |PARTITIONED|
+            exchange [cardinality: 1543.0, op-cost: 0.0, total-cost: 9716.46]
+            -- HASH_PARTITION_EXCHANGE [$$59]  |PARTITIONED|
+              group by ([$$59 := $$53]) decor ([]) {
+                        aggregate [$$58] <- [sql-count-serial(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- AGGREGATE  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                     } [cardinality: 1543.0, op-cost: 3711.46, total-cost: 9716.46]
+              -- EXTERNAL_GROUP_BY[$$53]  |PARTITIONED|
+                exchange [cardinality: 3711.46, op-cost: 0.0, total-cost: 6005.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  select (and(le($$54, 4), gt($$l.getField(4), 10))) project: [$$53] [cardinality: 3711.46, op-cost: 0.0, total-cost: 6005.0]
+                  -- STREAM_SELECT  |PARTITIONED|
+                    exchange [cardinality: 6005.0, op-cost: 0.0, total-cost: 6005.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$53, $$54, $$l] <- tpch.LineItem [cardinality: 6005.0, op-cost: 6005.0, total-cost: 6005.0]
+                      -- DATASOURCE_SCAN  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$53, $$54, $$l] <- tpch.LineItem [cardinality: 6005.0, op-cost: 6005.0, total-cost: 6005.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|
\ No newline at end of file
+                          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_cbo/cardinality-estimation/single-collection-queries/single-collection-queries.7.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/single-collection-queries/single-collection-queries.7.plan
index 7eb0d3f..6aff7f1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/single-collection-queries/single-collection-queries.7.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/cardinality-estimation/single-collection-queries/single-collection-queries.7.plan
@@ -2,47 +2,41 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 191.0, op-cost: 0.0, total-cost: 3965.67]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$51]) [cardinality: 191.0, op-cost: 0.0, total-cost: 3965.67]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$51] <- [{"$1": $$55}] [cardinality: 191.0, op-cost: 0.0, total-cost: 3965.67]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$55]) [cardinality: 191.0, op-cost: 0.0, total-cost: 3965.67]
-        -- STREAM_PROJECT  |PARTITIONED|
-          exchange [cardinality: 191.0, op-cost: 0.0, total-cost: 3965.67]
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            group by ([$$l_partkey := $$59]) decor ([]) {
-                      aggregate [$$55] <- [sql-sum-serial($$58)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- AGGREGATE  |LOCAL|
-                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                   } [cardinality: 191.0, op-cost: 1321.89, total-cost: 3965.67]
-            -- EXTERNAL_GROUP_BY[$$59]  |PARTITIONED|
-              exchange [cardinality: 191.0, op-cost: 0.0, total-cost: 2643.78]
-              -- HASH_PARTITION_EXCHANGE [$$59]  |PARTITIONED|
-                group by ([$$59 := $$52]) decor ([]) {
-                          aggregate [$$58] <- [sql-count-serial(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       } [cardinality: 191.0, op-cost: 1321.89, total-cost: 2643.78]
-                -- EXTERNAL_GROUP_BY[$$52]  |PARTITIONED|
-                  exchange [cardinality: 1321.89, op-cost: 0.0, total-cost: 1321.89]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$52]) [cardinality: 1321.89, op-cost: 0.0, total-cost: 1321.89]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      select (and(gt($$53, 1), gt($$54, 4))) [cardinality: 1321.89, op-cost: 0.0, total-cost: 1321.89]
-                      -- STREAM_SELECT  |PARTITIONED|
-                        project ([$$53, $$54, $$52]) [cardinality: 1321.89, op-cost: 0.0, total-cost: 1321.89]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$52] <- [$$l.getField(1)] [cardinality: 1321.89, op-cost: 0.0, total-cost: 1321.89]
-                          -- ASSIGN  |PARTITIONED|
-                            exchange [cardinality: 1321.89, op-cost: 0.0, total-cost: 1321.89]
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              unnest-map [$$53, $$54, $$l] <- index-search("LineItem", 0, "Default", "tpch", "LineItem", false, false, 2, $$60, $$61, 0, true, true, false) [cardinality: 1321.89, op-cost: 1321.89, total-cost: 1321.89]
-                              -- BTREE_SEARCH  |PARTITIONED|
-                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  assign [$$60, $$61] <- [1, 4] [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|
+    assign [$$51] <- [{"$1": $$55}] project: [$$51] [cardinality: 191.0, op-cost: 0.0, total-cost: 3965.67]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$55]) [cardinality: 191.0, op-cost: 0.0, total-cost: 3965.67]
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange [cardinality: 191.0, op-cost: 0.0, total-cost: 3965.67]
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          group by ([$$l_partkey := $$59]) decor ([]) {
+                    aggregate [$$55] <- [sql-sum-serial($$58)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                 } [cardinality: 191.0, op-cost: 1321.89, total-cost: 3965.67]
+          -- EXTERNAL_GROUP_BY[$$59]  |PARTITIONED|
+            exchange [cardinality: 191.0, op-cost: 0.0, total-cost: 2643.78]
+            -- HASH_PARTITION_EXCHANGE [$$59]  |PARTITIONED|
+              group by ([$$59 := $$52]) decor ([]) {
+                        aggregate [$$58] <- [sql-count-serial(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- AGGREGATE  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                     } [cardinality: 191.0, op-cost: 1321.89, total-cost: 2643.78]
+              -- EXTERNAL_GROUP_BY[$$52]  |PARTITIONED|
+                exchange [cardinality: 1321.89, op-cost: 0.0, total-cost: 1321.89]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  select (and(gt($$53, 1), gt($$54, 4))) project: [$$52] [cardinality: 1321.89, op-cost: 0.0, total-cost: 1321.89]
+                  -- STREAM_SELECT  |PARTITIONED|
+                    assign [$$52] <- [$$l.getField(1)] project: [$$53, $$54, $$52] [cardinality: 1321.89, op-cost: 0.0, total-cost: 1321.89]
+                    -- ASSIGN  |PARTITIONED|
+                      exchange [cardinality: 1321.89, op-cost: 0.0, total-cost: 1321.89]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        unnest-map [$$53, $$54, $$l] <- index-search("LineItem", 0, "Default", "tpch", "LineItem", false, false, 2, $$60, $$61, 0, true, true, false) [cardinality: 1321.89, op-cost: 1321.89, total-cost: 1321.89]
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            assign [$$60, $$61] <- [1, 4] [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_cbo/column/filter/001/001.007.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.007.plan
index 6443cbc..bbaca68 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.007.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.007.plan
@@ -10,17 +10,15 @@
         -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
           exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$20]) [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select (and(gt($$19, "1"), lt($$19, "3"))) [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                assign [$$19] <- [$$d.getField("a")] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                -- ASSIGN  |PARTITIONED|
-                  exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$20, $$d] <- test.ColumnDataset filter on: and(gt($$d.getField("a"), "1"), lt($$d.getField("a"), "3")) range-filter on: and(gt($$d.getField("a"), "1"), lt($$d.getField("a"), "3")) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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|
+            select (and(gt($$19, "1"), lt($$19, "3"))) project: [$$d, $$20] [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$19] <- [$$d.getField("a")] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+              -- ASSIGN  |PARTITIONED|
+                exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$20, $$d] <- test.ColumnDataset filter on: and(gt($$d.getField("a"), "1"), lt($$d.getField("a"), "3")) range-filter on: and(gt($$d.getField("a"), "1"), lt($$d.getField("a"), "3")) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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_cbo/column/filter/001/001.009.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.009.plan
index b1679c3..b9c50d6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.009.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.009.plan
@@ -10,17 +10,15 @@
         -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 6.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 6.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select (and(gt($$19, 1), lt($$19, 3))) [cardinality: 0.0, op-cost: 0.0, total-cost: 6.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                assign [$$19] <- [$$d.getField("a")] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                -- ASSIGN  |PARTITIONED|
-                  exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$20, $$d] <- test.ColumnDataset filter on: and(gt($$d.getField("a"), 1), lt($$d.getField("a"), 3)) range-filter on: and(gt($$d.getField("a"), 1), lt($$d.getField("a"), 3)) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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|
+            select (and(gt($$19, 1), lt($$19, 3))) project: [$$d, $$20] [cardinality: 0.0, op-cost: 0.0, total-cost: 6.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$19] <- [$$d.getField("a")] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+              -- ASSIGN  |PARTITIONED|
+                exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$20, $$d] <- test.ColumnDataset filter on: and(gt($$d.getField("a"), 1), lt($$d.getField("a"), 3)) range-filter on: and(gt($$d.getField("a"), 1), lt($$d.getField("a"), 3)) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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_cbo/column/filter/001/001.011.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.011.plan
index 7675958..cead9bc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.011.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.011.plan
@@ -10,30 +10,28 @@
         -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 6.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$28]) [cardinality: 0.0, op-cost: 0.0, total-cost: 6.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select ($$26) [cardinality: 0.0, op-cost: 0.0, total-cost: 6.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$28, $$d, $$26]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.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: 6.0, op-cost: 0.0, total-cost: 6.0]
-                  -- SUBPLAN  |PARTITIONED|
-                    assign [$$29] <- [$$d.getField("array")] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                    -- ASSIGN  |PARTITIONED|
-                      exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        data-scan []<-[$$28, $$d] <- test.ColumnDataset filter on: lt(scan-collection($$d.getField("array")), "100") range-filter on: lt(scan-collection($$d.getField("array")), "100") [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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|
+            select ($$26) project: [$$d, $$28] [cardinality: 0.0, op-cost: 0.0, total-cost: 6.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$28, $$d, $$26]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.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: 6.0, op-cost: 0.0, total-cost: 6.0]
+                -- SUBPLAN  |PARTITIONED|
+                  assign [$$29] <- [$$d.getField("array")] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                  -- ASSIGN  |PARTITIONED|
+                    exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$28, $$d] <- test.ColumnDataset filter on: lt(scan-collection($$d.getField("array")), "100") range-filter on: lt(scan-collection($$d.getField("array")), "100") [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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_cbo/column/filter/001/001.013.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.013.plan
index 51a1dee..766000b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.013.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.013.plan
@@ -10,30 +10,28 @@
         -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
           exchange [cardinality: 3.0, op-cost: 0.0, total-cost: 6.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$28]) [cardinality: 3.0, op-cost: 0.0, total-cost: 6.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select ($$26) [cardinality: 3.0, op-cost: 0.0, total-cost: 6.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$28, $$d, $$26]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.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: 6.0, op-cost: 0.0, total-cost: 6.0]
-                  -- SUBPLAN  |PARTITIONED|
-                    assign [$$29] <- [$$d.getField("array")] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                    -- ASSIGN  |PARTITIONED|
-                      exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        data-scan []<-[$$28, $$d] <- test.ColumnDataset filter on: lt(scan-collection($$d.getField("array")), 100) range-filter on: lt(scan-collection($$d.getField("array")), 100) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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|
+            select ($$26) project: [$$d, $$28] [cardinality: 3.0, op-cost: 0.0, total-cost: 6.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$28, $$d, $$26]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.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: 6.0, op-cost: 0.0, total-cost: 6.0]
+                -- SUBPLAN  |PARTITIONED|
+                  assign [$$29] <- [$$d.getField("array")] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                  -- ASSIGN  |PARTITIONED|
+                    exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$28, $$d] <- test.ColumnDataset filter on: lt(scan-collection($$d.getField("array")), 100) range-filter on: lt(scan-collection($$d.getField("array")), 100) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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_cbo/column/filter/001/001.015.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.015.plan
index 48b09a6..ac393cb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.015.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.015.plan
@@ -10,30 +10,28 @@
         -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
           exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$28]) [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select ($$26) [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$28, $$d, $$26]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.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: 6.0, op-cost: 0.0, total-cost: 6.0]
-                  -- SUBPLAN  |PARTITIONED|
-                    assign [$$29] <- [$$d.getField("array")] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                    -- ASSIGN  |PARTITIONED|
-                      exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        data-scan []<-[$$28, $$d] <- test.ColumnDataset filter on: eq(scan-collection($$d.getField("array")), 100) range-filter on: eq(scan-collection($$d.getField("array")), 100) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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|
+            select ($$26) project: [$$d, $$28] [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$28, $$d, $$26]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.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: 6.0, op-cost: 0.0, total-cost: 6.0]
+                -- SUBPLAN  |PARTITIONED|
+                  assign [$$29] <- [$$d.getField("array")] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                  -- ASSIGN  |PARTITIONED|
+                    exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$28, $$d] <- test.ColumnDataset filter on: eq(scan-collection($$d.getField("array")), 100) range-filter on: eq(scan-collection($$d.getField("array")), 100) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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_cbo/column/filter/001/001.017.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.017.plan
index 56d9e63..4cd5706 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.017.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.017.plan
@@ -10,30 +10,28 @@
         -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
           exchange [cardinality: 4.0, op-cost: 0.0, total-cost: 6.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$28]) [cardinality: 4.0, op-cost: 0.0, total-cost: 6.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select ($$26) [cardinality: 4.0, op-cost: 0.0, total-cost: 6.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$28, $$d, $$26]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.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: 6.0, op-cost: 0.0, total-cost: 6.0]
-                  -- SUBPLAN  |PARTITIONED|
-                    assign [$$29] <- [$$d.getField("array")] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                    -- ASSIGN  |PARTITIONED|
-                      exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        data-scan []<-[$$28, $$d] <- test.ColumnDataset filter on: lt(scan-collection($$d.getField("array")), 100.1) range-filter on: lt(scan-collection($$d.getField("array")), 100.1) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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|
+            select ($$26) project: [$$d, $$28] [cardinality: 4.0, op-cost: 0.0, total-cost: 6.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$28, $$d, $$26]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.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: 6.0, op-cost: 0.0, total-cost: 6.0]
+                -- SUBPLAN  |PARTITIONED|
+                  assign [$$29] <- [$$d.getField("array")] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                  -- ASSIGN  |PARTITIONED|
+                    exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$28, $$d] <- test.ColumnDataset filter on: lt(scan-collection($$d.getField("array")), 100.1) range-filter on: lt(scan-collection($$d.getField("array")), 100.1) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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_cbo/column/filter/001/001.019.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.019.plan
index 374eccc..81f715c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.019.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.019.plan
@@ -1,30 +1,26 @@
-distribute result [$$item] [cardinality: 3.0, op-cost: 0.0, total-cost: 22.75]
+distribute result [$$item] [cardinality: 3.0, op-cost: 0.0, total-cost: 10.75]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 3.0, op-cost: 0.0, total-cost: 22.75]
+  exchange [cardinality: 3.0, op-cost: 0.0, total-cost: 10.75]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$item]) [cardinality: 3.0, op-cost: 0.0, total-cost: 22.75]
+    project ([$$item]) [cardinality: 3.0, op-cost: 0.0, total-cost: 10.75]
     -- STREAM_PROJECT  |PARTITIONED|
-      exchange [cardinality: 3.0, op-cost: 0.0, total-cost: 22.75]
+      exchange [cardinality: 3.0, op-cost: 0.0, total-cost: 10.75]
       -- SORT_MERGE_EXCHANGE [$$29(ASC), $$item(ASC) ]  |PARTITIONED|
-        order (ASC, $$29) (ASC, $$item) [cardinality: 3.0, op-cost: 4.75, total-cost: 22.75]
+        order (ASC, $$29) (ASC, $$item) [cardinality: 3.0, op-cost: 4.75, total-cost: 10.75]
         -- STABLE_SORT [$$29(ASC), $$item(ASC)]  |PARTITIONED|
-          exchange [cardinality: 3.0, op-cost: 0.0, total-cost: 18.0]
+          exchange [cardinality: 3.0, op-cost: 0.0, total-cost: 6.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            select (and(gt($$item, 60), lt($$item, 100))) [cardinality: 3.0, op-cost: 0.0, total-cost: 18.0]
+            select (and(gt($$item, 60), lt($$item, 100))) [cardinality: 3.0, op-cost: 0.0, total-cost: 6.0]
             -- STREAM_SELECT  |PARTITIONED|
-              project ([$$29, $$item]) [cardinality: 18.0, op-cost: 0.0, total-cost: 18.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                unnest $$item <- scan-collection($$30) [cardinality: 18.0, op-cost: 0.0, total-cost: 18.0]
-                -- UNNEST  |PARTITIONED|
-                  project ([$$29, $$30]) [cardinality: 18.0, op-cost: 0.0, total-cost: 18.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$30] <- [$$d.getField("array")] [cardinality: 18.0, op-cost: 0.0, total-cost: 18.0]
-                    -- ASSIGN  |PARTITIONED|
-                      exchange [cardinality: 18.0, op-cost: 0.0, total-cost: 18.0]
+              unnest $$item <- scan-collection($$30) project: [$$29, $$item] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+              -- UNNEST  |PARTITIONED|
+                assign [$$30] <- [$$d.getField("array")] project: [$$29, $$30] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                -- ASSIGN  |PARTITIONED|
+                  exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$29, $$d] <- test.ColumnDataset project ({array:[any]}) filter on: and(gt(scan-collection($$d.getField("array")), 60), lt(scan-collection($$d.getField("array")), 100)) range-filter on: and(gt(scan-collection($$d.getField("array")), 60), lt(scan-collection($$d.getField("array")), 100)) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.0]
+                    -- DATASOURCE_SCAN  |PARTITIONED|
+                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        data-scan []<-[$$29, $$d] <- test.ColumnDataset project ({array:[any]}) filter on: and(gt(scan-collection($$d.getField("array")), 60), lt(scan-collection($$d.getField("array")), 100)) range-filter on: and(gt(scan-collection($$d.getField("array")), 60), lt(scan-collection($$d.getField("array")), 100)) [cardinality: 18.0, op-cost: 18.0, total-cost: 18.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|
+                        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_cbo/column/filter/001/001.021.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.021.plan
index d767732..df8c5bf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.021.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.021.plan
@@ -1,30 +1,26 @@
-distribute result [$$item] [cardinality: 1.0, op-cost: 0.0, total-cost: 18.0]
+distribute result [$$item] [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 18.0]
+  exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$item]) [cardinality: 1.0, op-cost: 0.0, total-cost: 18.0]
+    project ([$$item]) [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
     -- STREAM_PROJECT  |PARTITIONED|
-      exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 18.0]
+      exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
       -- SORT_MERGE_EXCHANGE [$$27(ASC), $$item(ASC) ]  |PARTITIONED|
-        order (ASC, $$27) (ASC, $$item) [cardinality: 1.0, op-cost: 0.0, total-cost: 18.0]
+        order (ASC, $$27) (ASC, $$item) [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
         -- STABLE_SORT [$$27(ASC), $$item(ASC)]  |PARTITIONED|
-          exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 18.0]
+          exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            select (eq($$item, 100)) [cardinality: 1.0, op-cost: 0.0, total-cost: 18.0]
+            select (eq($$item, 100)) [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
             -- STREAM_SELECT  |PARTITIONED|
-              project ([$$27, $$item]) [cardinality: 18.0, op-cost: 0.0, total-cost: 18.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                unnest $$item <- scan-collection($$28) [cardinality: 18.0, op-cost: 0.0, total-cost: 18.0]
-                -- UNNEST  |PARTITIONED|
-                  project ([$$27, $$28]) [cardinality: 18.0, op-cost: 0.0, total-cost: 18.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$28] <- [$$d.getField("array")] [cardinality: 18.0, op-cost: 0.0, total-cost: 18.0]
-                    -- ASSIGN  |PARTITIONED|
-                      exchange [cardinality: 18.0, op-cost: 0.0, total-cost: 18.0]
+              unnest $$item <- scan-collection($$28) project: [$$27, $$item] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+              -- UNNEST  |PARTITIONED|
+                assign [$$28] <- [$$d.getField("array")] project: [$$27, $$28] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                -- ASSIGN  |PARTITIONED|
+                  exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$27, $$d] <- test.ColumnDataset project ({array:[any]}) filter on: eq(scan-collection($$d.getField("array")), 100) range-filter on: eq(scan-collection($$d.getField("array")), 100) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.0]
+                    -- DATASOURCE_SCAN  |PARTITIONED|
+                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        data-scan []<-[$$27, $$d] <- test.ColumnDataset project ({array:[any]}) filter on: eq(scan-collection($$d.getField("array")), 100) range-filter on: eq(scan-collection($$d.getField("array")), 100) [cardinality: 18.0, op-cost: 18.0, total-cost: 18.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|
+                        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_cbo/column/filter/001/001.023.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.023.plan
index 7ab796d..ae4a534 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.023.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.023.plan
@@ -1,30 +1,26 @@
-distribute result [$$item] [cardinality: 0.0, op-cost: 0.0, total-cost: 18.0]
+distribute result [$$item] [cardinality: 0.0, op-cost: 0.0, total-cost: 6.0]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 18.0]
+  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 6.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$item]) [cardinality: 0.0, op-cost: 0.0, total-cost: 18.0]
+    project ([$$item]) [cardinality: 0.0, op-cost: 0.0, total-cost: 6.0]
     -- STREAM_PROJECT  |PARTITIONED|
-      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 18.0]
+      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 6.0]
       -- SORT_MERGE_EXCHANGE [$$27(ASC), $$item(ASC) ]  |PARTITIONED|
-        order (ASC, $$27) (ASC, $$item) [cardinality: 0.0, op-cost: 0.0, total-cost: 18.0]
+        order (ASC, $$27) (ASC, $$item) [cardinality: 0.0, op-cost: 0.0, total-cost: 6.0]
         -- STABLE_SORT [$$27(ASC), $$item(ASC)]  |PARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 18.0]
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 6.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            select (gt($$item, 10000)) [cardinality: 0.0, op-cost: 0.0, total-cost: 18.0]
+            select (gt($$item, 10000)) [cardinality: 0.0, op-cost: 0.0, total-cost: 6.0]
             -- STREAM_SELECT  |PARTITIONED|
-              project ([$$27, $$item]) [cardinality: 18.0, op-cost: 0.0, total-cost: 18.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                unnest $$item <- scan-collection($$28) [cardinality: 18.0, op-cost: 0.0, total-cost: 18.0]
-                -- UNNEST  |PARTITIONED|
-                  project ([$$27, $$28]) [cardinality: 18.0, op-cost: 0.0, total-cost: 18.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$28] <- [$$d.getField("array")] [cardinality: 18.0, op-cost: 0.0, total-cost: 18.0]
-                    -- ASSIGN  |PARTITIONED|
-                      exchange [cardinality: 18.0, op-cost: 0.0, total-cost: 18.0]
+              unnest $$item <- scan-collection($$28) project: [$$27, $$item] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+              -- UNNEST  |PARTITIONED|
+                assign [$$28] <- [$$d.getField("array")] project: [$$27, $$28] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                -- ASSIGN  |PARTITIONED|
+                  exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$27, $$d] <- test.ColumnDataset project ({array:[any]}) filter on: gt(scan-collection($$d.getField("array")), 10000) range-filter on: gt(scan-collection($$d.getField("array")), 10000) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.0]
+                    -- DATASOURCE_SCAN  |PARTITIONED|
+                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        data-scan []<-[$$27, $$d] <- test.ColumnDataset project ({array:[any]}) filter on: gt(scan-collection($$d.getField("array")), 10000) range-filter on: gt(scan-collection($$d.getField("array")), 10000) [cardinality: 18.0, op-cost: 18.0, total-cost: 18.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|
+                        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_cbo/column/filter/001/001.025.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.025.plan
index a62c529..19dacd2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.025.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/001/001.025.plan
@@ -1,36 +1,30 @@
-distribute result [$$31] [cardinality: 2.0, op-cost: 0.0, total-cost: 20.0]
+distribute result [$$31] [cardinality: 2.0, op-cost: 0.0, total-cost: 8.0]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 20.0]
+  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 8.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$31]) [cardinality: 2.0, op-cost: 0.0, total-cost: 20.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$31] <- [{"a": $$34, "item": $$item}] [cardinality: 2.0, op-cost: 0.0, total-cost: 20.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$34, $$item]) [cardinality: 2.0, op-cost: 0.0, total-cost: 20.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 20.0]
-          -- SORT_MERGE_EXCHANGE [$$35(ASC), $$34(ASC), $$item(ASC) ]  |PARTITIONED|
-            order (ASC, $$35) (ASC, $$34) (ASC, $$item) [cardinality: 2.0, op-cost: 2.0, total-cost: 20.0]
-            -- STABLE_SORT [$$35(ASC), $$34(ASC), $$item(ASC)]  |PARTITIONED|
-              exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 18.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                select (gt($$item, 10)) [cardinality: 2.0, op-cost: 0.0, total-cost: 18.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  project ([$$35, $$34, $$item]) [cardinality: 18.0, op-cost: 0.0, total-cost: 18.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    unnest $$item <- scan-collection($$36) [cardinality: 18.0, op-cost: 0.0, total-cost: 18.0]
-                    -- UNNEST  |PARTITIONED|
-                      select (eq($$34, "1")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_SELECT  |PARTITIONED|
-                        project ([$$35, $$34, $$36]) [cardinality: 18.0, op-cost: 0.0, total-cost: 18.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$34, $$36] <- [$$d.getField("a"), $$d.getField("array")] [cardinality: 18.0, op-cost: 0.0, total-cost: 18.0]
-                          -- ASSIGN  |PARTITIONED|
-                            exchange [cardinality: 18.0, op-cost: 0.0, total-cost: 18.0]
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              data-scan []<-[$$35, $$d] <- test.ColumnDataset project ({a:any,array:[any]}) filter on: and(eq($$d.getField("a"), "1"), gt(scan-collection($$d.getField("array")), 10)) range-filter on: and(eq($$d.getField("a"), "1"), gt(scan-collection($$d.getField("array")), 10)) [cardinality: 18.0, op-cost: 18.0, total-cost: 18.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|
+    assign [$$31] <- [{"a": $$34, "item": $$item}] project: [$$31] [cardinality: 2.0, op-cost: 0.0, total-cost: 8.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$34, $$item]) [cardinality: 2.0, op-cost: 0.0, total-cost: 8.0]
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 8.0]
+        -- SORT_MERGE_EXCHANGE [$$35(ASC), $$34(ASC), $$item(ASC) ]  |PARTITIONED|
+          order (ASC, $$35) (ASC, $$34) (ASC, $$item) [cardinality: 2.0, op-cost: 2.0, total-cost: 8.0]
+          -- STABLE_SORT [$$35(ASC), $$34(ASC), $$item(ASC)]  |PARTITIONED|
+            exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 6.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              select (gt($$item, 10)) [cardinality: 2.0, op-cost: 0.0, total-cost: 6.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                unnest $$item <- scan-collection($$36) project: [$$35, $$34, $$item] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                -- UNNEST  |PARTITIONED|
+                  select (eq($$34, "1")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_SELECT  |PARTITIONED|
+                    assign [$$34, $$36] <- [$$d.getField("a"), $$d.getField("array")] project: [$$35, $$34, $$36] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                    -- ASSIGN  |PARTITIONED|
+                      exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        data-scan []<-[$$35, $$d] <- test.ColumnDataset project ({a:any,array:[any]}) filter on: and(eq($$d.getField("a"), "1"), gt(scan-collection($$d.getField("array")), 10)) range-filter on: and(eq($$d.getField("a"), "1"), gt(scan-collection($$d.getField("array")), 10)) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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_cbo/column/filter/004/004.006.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/004/004.006.plan
index 11bdf61..f4faeeb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/004/004.006.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/004/004.006.plan
@@ -1,34 +1,28 @@
-distribute result [$$47] [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
+distribute result [$$47] [cardinality: 99.0, op-cost: 0.0, total-cost: 30.0]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
-  exchange [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
+  exchange [cardinality: 99.0, op-cost: 0.0, total-cost: 30.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$47]) [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$47] <- [{"$1": $$49}] [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$49] <- [agg-sql-sum($$51)] [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$51] <- [agg-sql-count(1)] [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
-            -- AGGREGATE  |PARTITIONED|
-              select (gt($$D, " ")) [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$D]) [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  unnest $$D <- scan-collection($$50) [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
-                  -- UNNEST  |PARTITIONED|
-                    project ([$$50]) [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$50] <- [$$C.getField("dates")] [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
-                      -- ASSIGN  |PARTITIONED|
-                        project ([$$C]) [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          exchange [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            data-scan []<-[$$48, $$C] <- TestYelp.YelpCheckin project ({dates:[any]}) filter on: gt(scan-collection($$C.getField("dates")), " ") range-filter on: gt(scan-collection($$C.getField("dates")), " ") [cardinality: 99.0, op-cost: 99.0, total-cost: 99.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|
+    assign [$$47] <- [{"$1": $$49}] project: [$$47] [cardinality: 99.0, op-cost: 0.0, total-cost: 30.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$49] <- [agg-sql-sum($$51)] [cardinality: 99.0, op-cost: 0.0, total-cost: 30.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 99.0, op-cost: 0.0, total-cost: 30.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$51] <- [agg-sql-count(1)] [cardinality: 99.0, op-cost: 0.0, total-cost: 30.0]
+          -- AGGREGATE  |PARTITIONED|
+            select (gt($$D, " ")) [cardinality: 99.0, op-cost: 0.0, total-cost: 30.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              unnest $$D <- scan-collection($$50) project: [$$D] [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+              -- UNNEST  |PARTITIONED|
+                assign [$$50] <- [$$C.getField("dates")] project: [$$50] [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$C]) [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    exchange [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$48, $$C] <- TestYelp.YelpCheckin project ({dates:[any]}) filter on: gt(scan-collection($$C.getField("dates")), " ") range-filter on: gt(scan-collection($$C.getField("dates")), " ") [cardinality: 30.0, op-cost: 30.0, total-cost: 30.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_cbo/column/filter/004/004.102.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/004/004.102.plan
index 59da462..c4eddc4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/004/004.102.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/004/004.102.plan
@@ -1,34 +1,28 @@
-distribute result [$$48] [cardinality: 6.0, op-cost: 0.0, total-cost: 101.0]
+distribute result [$$48] [cardinality: 6.0, op-cost: 0.0, total-cost: 30.0]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
-  exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 101.0]
+  exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 30.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$48]) [cardinality: 6.0, op-cost: 0.0, total-cost: 101.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$48] <- [{"$1": $$50}] [cardinality: 6.0, op-cost: 0.0, total-cost: 101.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$50] <- [agg-sql-sum($$52)] [cardinality: 6.0, op-cost: 0.0, total-cost: 101.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 101.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$52] <- [agg-sql-count(1)] [cardinality: 6.0, op-cost: 0.0, total-cost: 101.0]
-            -- AGGREGATE  |PARTITIONED|
-              select (eq(substring($$D, 0, 4), "2011")) [cardinality: 6.0, op-cost: 0.0, total-cost: 101.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$D]) [cardinality: 101.0, op-cost: 0.0, total-cost: 101.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  unnest $$D <- scan-collection($$51) [cardinality: 101.0, op-cost: 0.0, total-cost: 101.0]
-                  -- UNNEST  |PARTITIONED|
-                    project ([$$51]) [cardinality: 101.0, op-cost: 0.0, total-cost: 101.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$51] <- [$$C.getField("dates")] [cardinality: 101.0, op-cost: 0.0, total-cost: 101.0]
-                      -- ASSIGN  |PARTITIONED|
-                        project ([$$C]) [cardinality: 101.0, op-cost: 0.0, total-cost: 101.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          exchange [cardinality: 101.0, op-cost: 0.0, total-cost: 101.0]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            data-scan []<-[$$49, $$C] <- TestYelp.YelpCheckin project ({dates:[any]}) filter on: eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2011") [cardinality: 101.0, op-cost: 101.0, total-cost: 101.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|
+    assign [$$48] <- [{"$1": $$50}] project: [$$48] [cardinality: 6.0, op-cost: 0.0, total-cost: 30.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$50] <- [agg-sql-sum($$52)] [cardinality: 6.0, op-cost: 0.0, total-cost: 30.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 30.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$52] <- [agg-sql-count(1)] [cardinality: 6.0, op-cost: 0.0, total-cost: 30.0]
+          -- AGGREGATE  |PARTITIONED|
+            select (eq(substring($$D, 0, 4), "2011")) [cardinality: 6.0, op-cost: 0.0, total-cost: 30.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              unnest $$D <- scan-collection($$51) project: [$$D] [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+              -- UNNEST  |PARTITIONED|
+                assign [$$51] <- [$$C.getField("dates")] project: [$$51] [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$C]) [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    exchange [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$49, $$C] <- TestYelp.YelpCheckin project ({dates:[any]}) filter on: eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2011") [cardinality: 30.0, op-cost: 30.0, total-cost: 30.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_cbo/column/filter/004/004.105.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/004/004.105.plan
index 971b8d9..786d9da 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/004/004.105.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/004/004.105.plan
@@ -1,38 +1,30 @@
-distribute result [$$51] [cardinality: 20.0, op-cost: 0.0, total-cost: 101.0]
+distribute result [$$51] [cardinality: 20.0, op-cost: 0.0, total-cost: 30.0]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
-  exchange [cardinality: 20.0, op-cost: 0.0, total-cost: 101.0]
+  exchange [cardinality: 20.0, op-cost: 0.0, total-cost: 30.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$51]) [cardinality: 20.0, op-cost: 0.0, total-cost: 101.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$51] <- [{"$1": $$54}] [cardinality: 20.0, op-cost: 0.0, total-cost: 101.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$54] <- [agg-sql-sum($$57)] [cardinality: 20.0, op-cost: 0.0, total-cost: 101.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 20.0, op-cost: 0.0, total-cost: 101.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$57] <- [agg-sql-count(1)] [cardinality: 20.0, op-cost: 0.0, total-cost: 101.0]
-            -- AGGREGATE  |PARTITIONED|
-              select (or(eq($$58, "2011"), eq($$58, "2016"))) [cardinality: 20.0, op-cost: 0.0, total-cost: 101.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$58]) [cardinality: 101.0, op-cost: 0.0, total-cost: 101.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$58] <- [substring($$D, 0, 4)] [cardinality: 101.0, op-cost: 0.0, total-cost: 101.0]
+    assign [$$51] <- [{"$1": $$54}] project: [$$51] [cardinality: 20.0, op-cost: 0.0, total-cost: 30.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$54] <- [agg-sql-sum($$57)] [cardinality: 20.0, op-cost: 0.0, total-cost: 30.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 20.0, op-cost: 0.0, total-cost: 30.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$57] <- [agg-sql-count(1)] [cardinality: 20.0, op-cost: 0.0, total-cost: 30.0]
+          -- AGGREGATE  |PARTITIONED|
+            select (or(eq($$58, "2011"), eq($$58, "2016"))) [cardinality: 20.0, op-cost: 0.0, total-cost: 30.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$58] <- [substring($$D, 0, 4)] project: [$$58] [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+              -- ASSIGN  |PARTITIONED|
+                unnest $$D <- scan-collection($$55) project: [$$D] [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+                -- UNNEST  |PARTITIONED|
+                  assign [$$55] <- [$$C.getField("dates")] project: [$$55] [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
                   -- ASSIGN  |PARTITIONED|
-                    project ([$$D]) [cardinality: 101.0, op-cost: 0.0, total-cost: 101.0]
+                    project ([$$C]) [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
                     -- STREAM_PROJECT  |PARTITIONED|
-                      unnest $$D <- scan-collection($$55) [cardinality: 101.0, op-cost: 0.0, total-cost: 101.0]
-                      -- UNNEST  |PARTITIONED|
-                        project ([$$55]) [cardinality: 101.0, op-cost: 0.0, total-cost: 101.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$55] <- [$$C.getField("dates")] [cardinality: 101.0, op-cost: 0.0, total-cost: 101.0]
-                          -- ASSIGN  |PARTITIONED|
-                            project ([$$C]) [cardinality: 101.0, op-cost: 0.0, total-cost: 101.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              exchange [cardinality: 101.0, op-cost: 0.0, total-cost: 101.0]
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                data-scan []<-[$$53, $$C] <- TestYelp.YelpCheckin project ({dates:[any]}) filter on: or(eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2011"), eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2016")) [cardinality: 101.0, op-cost: 101.0, total-cost: 101.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: 30.0, op-cost: 0.0, total-cost: 30.0]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        data-scan []<-[$$53, $$C] <- TestYelp.YelpCheckin project ({dates:[any]}) filter on: or(eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2011"), eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2016")) [cardinality: 30.0, op-cost: 30.0, total-cost: 30.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_cbo/column/filter/004/004.108.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/004/004.108.plan
index d117252..edb62b1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/004/004.108.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/004/004.108.plan
@@ -2,44 +2,40 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 11.0, op-cost: 0.0, total-cost: 30.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$52]) [cardinality: 11.0, op-cost: 0.0, total-cost: 30.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$52] <- [{"$1": $$55}] [cardinality: 11.0, op-cost: 0.0, total-cost: 30.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$55] <- [agg-sql-sum($$58)] [cardinality: 11.0, op-cost: 0.0, total-cost: 30.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 11.0, op-cost: 0.0, total-cost: 30.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$58] <- [agg-sql-count(1)] [cardinality: 11.0, op-cost: 0.0, total-cost: 30.0]
-            -- AGGREGATE  |PARTITIONED|
-              select ($$45) [cardinality: 11.0, op-cost: 0.0, total-cost: 30.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$45]) [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  subplan {
-                            aggregate [$$45] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- AGGREGATE  |LOCAL|
-                              select (or(eq($$59, "2011"), eq($$59, "2016"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_SELECT  |LOCAL|
-                                assign [$$59] <- [substring($$D, 0, 4)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- ASSIGN  |LOCAL|
-                                  unnest $$D <- scan-collection($$56) [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: 30.0, op-cost: 0.0, total-cost: 30.0]
-                  -- SUBPLAN  |PARTITIONED|
-                    project ([$$56]) [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+    assign [$$52] <- [{"$1": $$55}] project: [$$52] [cardinality: 11.0, op-cost: 0.0, total-cost: 30.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$55] <- [agg-sql-sum($$58)] [cardinality: 11.0, op-cost: 0.0, total-cost: 30.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 11.0, op-cost: 0.0, total-cost: 30.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$58] <- [agg-sql-count(1)] [cardinality: 11.0, op-cost: 0.0, total-cost: 30.0]
+          -- AGGREGATE  |PARTITIONED|
+            select ($$45) [cardinality: 11.0, op-cost: 0.0, total-cost: 30.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$45]) [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+              -- STREAM_PROJECT  |PARTITIONED|
+                subplan {
+                          aggregate [$$45] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- AGGREGATE  |LOCAL|
+                            select (or(eq($$59, "2011"), eq($$59, "2016"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- STREAM_SELECT  |LOCAL|
+                              assign [$$59] <- [substring($$D, 0, 4)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- ASSIGN  |LOCAL|
+                                unnest $$D <- scan-collection($$56) [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: 30.0, op-cost: 0.0, total-cost: 30.0]
+                -- SUBPLAN  |PARTITIONED|
+                  assign [$$56] <- [$$C.getField("dates")] project: [$$56] [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+                  -- ASSIGN  |PARTITIONED|
+                    project ([$$C]) [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
                     -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$56] <- [$$C.getField("dates")] [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
-                      -- ASSIGN  |PARTITIONED|
-                        project ([$$C]) [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          exchange [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+                      exchange [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        data-scan []<-[$$54, $$C] <- TestYelp.YelpCheckin project ({dates:[any]}) filter on: or(eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2011"), eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2016")) [cardinality: 30.0, op-cost: 30.0, total-cost: 30.0]
+                        -- DATASOURCE_SCAN  |PARTITIONED|
+                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            data-scan []<-[$$54, $$C] <- TestYelp.YelpCheckin project ({dates:[any]}) filter on: or(eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2011"), eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2016")) [cardinality: 30.0, op-cost: 30.0, total-cost: 30.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|
+                            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_cbo/column/filter/004/004.113.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/004/004.113.plan
index 5254584..2f5f950 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/004/004.113.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/004/004.113.plan
@@ -1,40 +1,32 @@
-distribute result [$$54] [cardinality: 7.0, op-cost: 0.0, total-cost: 101.0]
+distribute result [$$54] [cardinality: 7.0, op-cost: 0.0, total-cost: 30.0]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
-  exchange [cardinality: 7.0, op-cost: 0.0, total-cost: 101.0]
+  exchange [cardinality: 7.0, op-cost: 0.0, total-cost: 30.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$54]) [cardinality: 7.0, op-cost: 0.0, total-cost: 101.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$54] <- [{"$1": $$57}] [cardinality: 7.0, op-cost: 0.0, total-cost: 101.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$57] <- [agg-sql-sum($$61)] [cardinality: 7.0, op-cost: 0.0, total-cost: 101.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 7.0, op-cost: 0.0, total-cost: 101.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$61] <- [agg-sql-count(1)] [cardinality: 7.0, op-cost: 0.0, total-cost: 101.0]
-            -- AGGREGATE  |PARTITIONED|
-              select (or(eq($$62, "2011"), eq($$62, "2016"))) [cardinality: 7.0, op-cost: 0.0, total-cost: 101.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$62]) [cardinality: 101.0, op-cost: 0.0, total-cost: 101.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$62] <- [substring($$D, 0, 4)] [cardinality: 101.0, op-cost: 0.0, total-cost: 101.0]
-                  -- ASSIGN  |PARTITIONED|
-                    project ([$$D]) [cardinality: 101.0, op-cost: 0.0, total-cost: 101.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      unnest $$D <- scan-collection($$59) [cardinality: 101.0, op-cost: 0.0, total-cost: 101.0]
-                      -- UNNEST  |PARTITIONED|
-                        project ([$$59]) [cardinality: 101.0, op-cost: 0.0, total-cost: 101.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          select (starts-with($$C.getField("business_id"), "-0")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- STREAM_SELECT  |PARTITIONED|
-                            assign [$$59] <- [$$C.getField("dates")] [cardinality: 101.0, op-cost: 0.0, total-cost: 101.0]
-                            -- ASSIGN  |PARTITIONED|
-                              project ([$$C]) [cardinality: 101.0, op-cost: 0.0, total-cost: 101.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                exchange [cardinality: 101.0, op-cost: 0.0, total-cost: 101.0]
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  data-scan []<-[$$56, $$C] <- TestYelp.YelpCheckin project ({dates:[any],business_id:any}) filter on: and(starts-with($$C.getField("business_id"), "-0"), or(eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2011"), eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2016"))) [cardinality: 101.0, op-cost: 101.0, total-cost: 101.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|
+    assign [$$54] <- [{"$1": $$57}] project: [$$54] [cardinality: 7.0, op-cost: 0.0, total-cost: 30.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$57] <- [agg-sql-sum($$61)] [cardinality: 7.0, op-cost: 0.0, total-cost: 30.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 7.0, op-cost: 0.0, total-cost: 30.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$61] <- [agg-sql-count(1)] [cardinality: 7.0, op-cost: 0.0, total-cost: 30.0]
+          -- AGGREGATE  |PARTITIONED|
+            select (or(eq($$62, "2011"), eq($$62, "2016"))) [cardinality: 7.0, op-cost: 0.0, total-cost: 30.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$62] <- [substring($$D, 0, 4)] project: [$$62] [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+              -- ASSIGN  |PARTITIONED|
+                unnest $$D <- scan-collection($$59) project: [$$D] [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+                -- UNNEST  |PARTITIONED|
+                  select (starts-with($$C.getField("business_id"), "-0")) project: [$$59] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_SELECT  |PARTITIONED|
+                    assign [$$59] <- [$$C.getField("dates")] [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+                    -- ASSIGN  |PARTITIONED|
+                      project ([$$C]) [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        exchange [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$56, $$C] <- TestYelp.YelpCheckin project ({dates:[any],business_id:any}) filter on: and(starts-with($$C.getField("business_id"), "-0"), or(eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2011"), eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2016"))) [cardinality: 30.0, op-cost: 30.0, total-cost: 30.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_cbo/column/filter/005/005.008.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/005/005.008.plan
index 95dc1df..06b8a09 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/005/005.008.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/005/005.008.plan
@@ -1,34 +1,28 @@
-distribute result [$$48] [cardinality: 2.0, op-cost: 0.0, total-cost: 99.0]
+distribute result [$$48] [cardinality: 2.0, op-cost: 0.0, total-cost: 30.0]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
-  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 99.0]
+  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 30.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$48]) [cardinality: 2.0, op-cost: 0.0, total-cost: 99.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$48] <- [{"$1": $$50}] [cardinality: 2.0, op-cost: 0.0, total-cost: 99.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$50] <- [agg-sql-sum($$52)] [cardinality: 2.0, op-cost: 0.0, total-cost: 99.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 99.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$52] <- [agg-sql-count(1)] [cardinality: 2.0, op-cost: 0.0, total-cost: 99.0]
-            -- AGGREGATE  |PARTITIONED|
-              select (gt(numeric-add($$D, 1), 2018)) [cardinality: 2.0, op-cost: 0.0, total-cost: 99.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$D]) [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  unnest $$D <- scan-collection($$51) [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
-                  -- UNNEST  |PARTITIONED|
-                    project ([$$51]) [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$51] <- [$$C.getField("dates")] [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
-                      -- ASSIGN  |PARTITIONED|
-                        project ([$$C]) [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          exchange [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            data-scan []<-[$$49, $$C] <- TestYelp.YelpCheckin project ({dates:[any]}) filter on: gt(numeric-add(scan-collection($$C.getField("dates")), 1), 2018) [cardinality: 99.0, op-cost: 99.0, total-cost: 99.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|
+    assign [$$48] <- [{"$1": $$50}] project: [$$48] [cardinality: 2.0, op-cost: 0.0, total-cost: 30.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$50] <- [agg-sql-sum($$52)] [cardinality: 2.0, op-cost: 0.0, total-cost: 30.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 30.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$52] <- [agg-sql-count(1)] [cardinality: 2.0, op-cost: 0.0, total-cost: 30.0]
+          -- AGGREGATE  |PARTITIONED|
+            select (gt(numeric-add($$D, 1), 2018)) [cardinality: 2.0, op-cost: 0.0, total-cost: 30.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              unnest $$D <- scan-collection($$51) project: [$$D] [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+              -- UNNEST  |PARTITIONED|
+                assign [$$51] <- [$$C.getField("dates")] project: [$$51] [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$C]) [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    exchange [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$49, $$C] <- TestYelp.YelpCheckin project ({dates:[any]}) filter on: gt(numeric-add(scan-collection($$C.getField("dates")), 1), 2018) [cardinality: 30.0, op-cost: 30.0, total-cost: 30.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_cbo/column/filter/005/005.011.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/005/005.011.plan
index 40ac636..e88ae19 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/005/005.011.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/005/005.011.plan
@@ -1,34 +1,28 @@
-distribute result [$$51] [cardinality: 8.0, op-cost: 0.0, total-cost: 99.0]
+distribute result [$$51] [cardinality: 8.0, op-cost: 0.0, total-cost: 30.0]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
-  exchange [cardinality: 8.0, op-cost: 0.0, total-cost: 99.0]
+  exchange [cardinality: 8.0, op-cost: 0.0, total-cost: 30.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$51]) [cardinality: 8.0, op-cost: 0.0, total-cost: 99.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$51] <- [{"$1": $$53}] [cardinality: 8.0, op-cost: 0.0, total-cost: 99.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$53] <- [agg-sql-sum($$55)] [cardinality: 8.0, op-cost: 0.0, total-cost: 99.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 8.0, op-cost: 0.0, total-cost: 99.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$55] <- [agg-sql-count(1)] [cardinality: 8.0, op-cost: 0.0, total-cost: 99.0]
-            -- AGGREGATE  |PARTITIONED|
-              select (or(gt(numeric-add($$D, 1), 2018), eq(substring($$D, 0, 4), "2011"))) [cardinality: 8.0, op-cost: 0.0, total-cost: 99.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$D]) [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  unnest $$D <- scan-collection($$54) [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
-                  -- UNNEST  |PARTITIONED|
-                    project ([$$54]) [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$54] <- [$$C.getField("dates")] [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
-                      -- ASSIGN  |PARTITIONED|
-                        project ([$$C]) [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          exchange [cardinality: 99.0, op-cost: 0.0, total-cost: 99.0]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            data-scan []<-[$$52, $$C] <- TestYelp.YelpCheckin project ({dates:[any]}) filter on: or(gt(numeric-add(scan-collection($$C.getField("dates")), 1), 2018), eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2011")) [cardinality: 99.0, op-cost: 99.0, total-cost: 99.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|
+    assign [$$51] <- [{"$1": $$53}] project: [$$51] [cardinality: 8.0, op-cost: 0.0, total-cost: 30.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$53] <- [agg-sql-sum($$55)] [cardinality: 8.0, op-cost: 0.0, total-cost: 30.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 8.0, op-cost: 0.0, total-cost: 30.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$55] <- [agg-sql-count(1)] [cardinality: 8.0, op-cost: 0.0, total-cost: 30.0]
+          -- AGGREGATE  |PARTITIONED|
+            select (or(gt(numeric-add($$D, 1), 2018), eq(substring($$D, 0, 4), "2011"))) [cardinality: 8.0, op-cost: 0.0, total-cost: 30.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              unnest $$D <- scan-collection($$54) project: [$$D] [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+              -- UNNEST  |PARTITIONED|
+                assign [$$54] <- [$$C.getField("dates")] project: [$$54] [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$C]) [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    exchange [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$52, $$C] <- TestYelp.YelpCheckin project ({dates:[any]}) filter on: or(gt(numeric-add(scan-collection($$C.getField("dates")), 1), 2018), eq(substring(scan-collection($$C.getField("dates")), 0, 4), "2011")) [cardinality: 30.0, op-cost: 30.0, total-cost: 30.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_cbo/column/filter/006/006.010.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/006/006.010.plan
index 73b314b..a6ad325 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/006/006.010.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/006/006.010.plan
@@ -2,44 +2,40 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 8.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$51]) [cardinality: 1.0, op-cost: 0.0, total-cost: 8.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$51] <- [{"$1": $$53}] [cardinality: 1.0, op-cost: 0.0, total-cost: 8.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$53] <- [agg-sql-sum($$57)] [cardinality: 1.0, op-cost: 0.0, total-cost: 8.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 8.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$57] <- [agg-sql-count(1)] [cardinality: 1.0, op-cost: 0.0, total-cost: 8.0]
-            -- AGGREGATE  |PARTITIONED|
-              select ($$44) [cardinality: 1.0, op-cost: 0.0, total-cost: 8.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$44]) [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  subplan {
-                            aggregate [$$44] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- AGGREGATE  |LOCAL|
-                              select (eq($$55, array: [ 5.1 ])) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_SELECT  |LOCAL|
-                                assign [$$55] <- [$$point.getField("lon")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- ASSIGN  |LOCAL|
-                                  unnest $$point <- scan-collection($$54) [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: 8.0, op-cost: 0.0, total-cost: 8.0]
-                  -- SUBPLAN  |PARTITIONED|
-                    project ([$$54]) [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
+    assign [$$51] <- [{"$1": $$53}] project: [$$51] [cardinality: 1.0, op-cost: 0.0, total-cost: 8.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$53] <- [agg-sql-sum($$57)] [cardinality: 1.0, op-cost: 0.0, total-cost: 8.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 8.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$57] <- [agg-sql-count(1)] [cardinality: 1.0, op-cost: 0.0, total-cost: 8.0]
+          -- AGGREGATE  |PARTITIONED|
+            select ($$44) [cardinality: 1.0, op-cost: 0.0, total-cost: 8.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$44]) [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
+              -- STREAM_PROJECT  |PARTITIONED|
+                subplan {
+                          aggregate [$$44] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- AGGREGATE  |LOCAL|
+                            select (eq($$55, array: [ 5.1 ])) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- STREAM_SELECT  |LOCAL|
+                              assign [$$55] <- [$$point.getField("lon")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- ASSIGN  |LOCAL|
+                                unnest $$point <- scan-collection($$54) [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: 8.0, op-cost: 0.0, total-cost: 8.0]
+                -- SUBPLAN  |PARTITIONED|
+                  assign [$$54] <- [$$c.getField("geo").getField("coordinates")] project: [$$54] [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
+                  -- ASSIGN  |PARTITIONED|
+                    project ([$$c]) [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
                     -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$54] <- [$$c.getField("geo").getField("coordinates")] [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
-                      -- ASSIGN  |PARTITIONED|
-                        project ([$$c]) [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          exchange [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
+                      exchange [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        data-scan []<-[$$52, $$c] <- test.ColumnDataset project ({geo:{coordinates:[{lon:any}]}}) [cardinality: 8.0, op-cost: 8.0, total-cost: 8.0]
+                        -- DATASOURCE_SCAN  |PARTITIONED|
+                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            data-scan []<-[$$52, $$c] <- test.ColumnDataset project ({geo:{coordinates:[{lon:any}]}}) [cardinality: 8.0, op-cost: 8.0, total-cost: 8.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|
+                            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_cbo/column/filter/007/007.010.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/007/007.010.plan
index dfc23ab..7b50535 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/007/007.010.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/007/007.010.plan
@@ -2,25 +2,23 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 30.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$37]) [cardinality: 1.0, op-cost: 0.0, total-cost: 30.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$37] <- [{"$1": $$39}] [cardinality: 1.0, op-cost: 0.0, total-cost: 30.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$39] <- [agg-sql-sum($$42)] [cardinality: 1.0, op-cost: 0.0, total-cost: 30.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 30.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$42] <- [agg-sql-count(1)] [cardinality: 1.0, op-cost: 0.0, total-cost: 30.0]
-            -- AGGREGATE  |PARTITIONED|
-              select (array-contains($$C.getField("checkin_times").getField("dates"), "2018-05-25")) [cardinality: 1.0, op-cost: 0.0, total-cost: 30.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$C]) [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  exchange [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$38, $$C] <- TestYelp.YelpCheckin project ({checkin_times:{dates:any}}) [cardinality: 30.0, op-cost: 30.0, total-cost: 30.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|
+    assign [$$37] <- [{"$1": $$39}] project: [$$37] [cardinality: 1.0, op-cost: 0.0, total-cost: 30.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$39] <- [agg-sql-sum($$42)] [cardinality: 1.0, op-cost: 0.0, total-cost: 30.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 30.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$42] <- [agg-sql-count(1)] [cardinality: 1.0, op-cost: 0.0, total-cost: 30.0]
+          -- AGGREGATE  |PARTITIONED|
+            select (array-contains($$C.getField("checkin_times").getField("dates"), "2018-05-25")) [cardinality: 1.0, op-cost: 0.0, total-cost: 30.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$C]) [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+              -- STREAM_PROJECT  |PARTITIONED|
+                exchange [cardinality: 30.0, op-cost: 0.0, total-cost: 30.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$38, $$C] <- TestYelp.YelpCheckin project ({checkin_times:{dates:any}}) [cardinality: 30.0, op-cost: 30.0, total-cost: 30.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_cbo/column/filter/not-in_every/not-in_every.101.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/not-in_every/not-in_every.101.plan
index c523b20..7650418 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/not-in_every/not-in_every.101.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/not-in_every/not-in_every.101.plan
@@ -10,28 +10,26 @@
         -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
           exchange [cardinality: 5.0, op-cost: 0.0, total-cost: 6.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$20]) [cardinality: 5.0, op-cost: 0.0, total-cost: 6.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select ($$18) [cardinality: 5.0, op-cost: 0.0, total-cost: 6.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$20, $$d, $$18]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  subplan {
-                            aggregate [$$18] <- [empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- AGGREGATE  |LOCAL|
-                              select (not(if-missing-or-null(neq($$21, "1"), false))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_SELECT  |LOCAL|
-                                nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                         } [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                  -- SUBPLAN  |PARTITIONED|
-                    assign [$$21] <- [$$d.getField("a")] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                    -- ASSIGN  |PARTITIONED|
-                      exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        data-scan []<-[$$20, $$d] <- test.ColumnDataset [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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|
+            select ($$18) project: [$$d, $$20] [cardinality: 5.0, op-cost: 0.0, total-cost: 6.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$20, $$d, $$18]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+              -- STREAM_PROJECT  |PARTITIONED|
+                subplan {
+                          aggregate [$$18] <- [empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- AGGREGATE  |LOCAL|
+                            select (not(if-missing-or-null(neq($$21, "1"), false))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- STREAM_SELECT  |LOCAL|
+                              nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                       } [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                -- SUBPLAN  |PARTITIONED|
+                  assign [$$21] <- [$$d.getField("a")] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                  -- ASSIGN  |PARTITIONED|
+                    exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$20, $$d] <- test.ColumnDataset [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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_cbo/column/filter/not-in_every/not-in_every.201.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/not-in_every/not-in_every.201.plan
index 20294f4..d4ae3fa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/not-in_every/not-in_every.201.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/not-in_every/not-in_every.201.plan
@@ -1,55 +1,53 @@
-distribute result [$$d] [cardinality: 3.0, op-cost: 0.0, total-cost: 15.5]
+distribute result [$$d] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 3.0, op-cost: 0.0, total-cost: 15.5]
+  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$d]) [cardinality: 3.0, op-cost: 0.0, total-cost: 15.5]
+    project ([$$d]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
     -- STREAM_PROJECT  |PARTITIONED|
-      exchange [cardinality: 3.0, op-cost: 0.0, total-cost: 15.5]
+      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-        project ([$$d, $$24]) [cardinality: 3.0, op-cost: 0.0, total-cost: 15.5]
-        -- STREAM_PROJECT  |PARTITIONED|
-          select ($$18) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STREAM_SELECT  |PARTITIONED|
-            exchange [cardinality: 3.0, op-cost: 0.0, total-cost: 15.5]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              group by ([$$24 := $$20]) decor ([$$d]) {
-                        aggregate [$$18] <- [empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          select (not(is-missing($$23))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- STREAM_SELECT  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 3.0, op-cost: 4.75, total-cost: 15.5]
-              -- PRE_CLUSTERED_GROUP_BY[$$20]  |PARTITIONED|
-                exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 10.75]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  order (ASC, $$20) [cardinality: 6.0, op-cost: 4.75, total-cost: 10.75]
-                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-                    exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                    -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
-                      project ([$$d, $$23, $$20]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          left outer join (not(if-missing-or-null(neq($$21, $#1), false))) [cardinality: 3.0, op-cost: 12.6, total-cost: 23.6]
-                          -- NESTED_LOOP  |PARTITIONED|
-                            exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              assign [$$21] <- [$$d.getField("a")] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                              -- ASSIGN  |PARTITIONED|
-                                exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  data-scan []<-[$$20, $$d] <- test.ColumnDataset [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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]
-                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                              assign [$$23] <- [true] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ASSIGN  |UNPARTITIONED|
-                                unnest $#1 <- scan-collection(array: [ "1", "2", "3" ]) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.0]
-                                -- UNNEST  |UNPARTITIONED|
-                                  empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+        select ($$18) project: [$$d, $$24] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STREAM_SELECT  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            group by ([$$24 := $$20]) decor ([$$d]) {
+                      aggregate [$$18] <- [empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        select (not(is-missing($$23))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- STREAM_SELECT  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- PRE_CLUSTERED_GROUP_BY[$$20]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                order (ASC, $$20) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
+                    project ([$$d, $$23, $$20]) [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 (not(if-missing-or-null(neq($$21, $#1), false))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_LOOP  |PARTITIONED|
+                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            assign [$$21] <- [$$d.getField("a")] [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]
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                data-scan []<-[$$20, $$d] <- test.ColumnDataset [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]
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            assign [$$23] <- [true] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- ASSIGN  |UNPARTITIONED|
+                              unnest $#1 <- scan-collection(array: [ "1", "2", "3" ]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- UNNEST  |UNPARTITIONED|
+                                empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/not-in_every/not-in_every.301.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/not-in_every/not-in_every.301.plan
index ec838b6..681ae89 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/not-in_every/not-in_every.301.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/not-in_every/not-in_every.301.plan
@@ -10,30 +10,28 @@
         -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
           exchange [cardinality: 3.0, op-cost: 0.0, total-cost: 6.0]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$d, $$28]) [cardinality: 3.0, op-cost: 0.0, total-cost: 6.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              select ($$26) [cardinality: 3.0, op-cost: 0.0, total-cost: 6.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$28, $$d, $$26]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  subplan {
-                            aggregate [$$26] <- [empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- AGGREGATE  |LOCAL|
-                              select (not(if-missing-or-null(ge($$x, 100), false))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_SELECT  |LOCAL|
-                                unnest $$x <- 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: 6.0, op-cost: 0.0, total-cost: 6.0]
-                  -- SUBPLAN  |PARTITIONED|
-                    assign [$$29] <- [$$d.getField("array")] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                    -- ASSIGN  |PARTITIONED|
-                      exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        data-scan []<-[$$28, $$d] <- test.ColumnDataset [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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|
+            select ($$26) project: [$$d, $$28] [cardinality: 3.0, op-cost: 0.0, total-cost: 6.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$28, $$d, $$26]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+              -- STREAM_PROJECT  |PARTITIONED|
+                subplan {
+                          aggregate [$$26] <- [empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- AGGREGATE  |LOCAL|
+                            select (not(if-missing-or-null(ge($$x, 100), false))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- STREAM_SELECT  |LOCAL|
+                              unnest $$x <- 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: 6.0, op-cost: 0.0, total-cost: 6.0]
+                -- SUBPLAN  |PARTITIONED|
+                  assign [$$29] <- [$$d.getField("array")] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                  -- ASSIGN  |PARTITIONED|
+                    exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$28, $$d] <- test.ColumnDataset [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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_cbo/column/filter/subplan/subplan.012.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/subplan/subplan.012.plan
index 81a8ea1..8cbbb97 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/subplan/subplan.012.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/subplan/subplan.012.plan
@@ -23,17 +23,15 @@
                               -- NESTED_TUPLE_SOURCE  |LOCAL|
                    } [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
             -- SUBPLAN  |PARTITIONED|
-              project ([$$57]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                assign [$$57] <- [$$p.getField("arrayOrObject")] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                -- ASSIGN  |PARTITIONED|
-                  project ([$$p]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      data-scan []<-[$$52, $$p] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) filter on: or(eq($$p.getField("arrayOrObject").getField("text"), "7"), eq(scan-collection($$p.getField("arrayOrObject")).getField("text"), "1")) range-filter on: or(eq($$p.getField("arrayOrObject").getField("text"), "7"), eq(scan-collection($$p.getField("arrayOrObject")).getField("text"), "1")) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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|
+              assign [$$57] <- [$$p.getField("arrayOrObject")] project: [$$57] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+              -- ASSIGN  |PARTITIONED|
+                project ([$$p]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                -- STREAM_PROJECT  |PARTITIONED|
+                  exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$52, $$p] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) filter on: or(eq($$p.getField("arrayOrObject").getField("text"), "7"), eq(scan-collection($$p.getField("arrayOrObject")).getField("text"), "1")) range-filter on: or(eq($$p.getField("arrayOrObject").getField("text"), "7"), eq(scan-collection($$p.getField("arrayOrObject")).getField("text"), "1")) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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_cbo/column/filter/subplan/subplan.022.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/subplan/subplan.022.plan
index 7f6ceee..75b8051 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/subplan/subplan.022.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/subplan/subplan.022.plan
@@ -23,17 +23,15 @@
                               -- NESTED_TUPLE_SOURCE  |LOCAL|
                    } [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
             -- SUBPLAN  |PARTITIONED|
-              project ([$$57]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                assign [$$57] <- [$$p.getField("arrayOrObject")] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                -- ASSIGN  |PARTITIONED|
-                  project ([$$p]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      data-scan []<-[$$52, $$p] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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|
+              assign [$$57] <- [$$p.getField("arrayOrObject")] project: [$$57] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+              -- ASSIGN  |PARTITIONED|
+                project ([$$p]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                -- STREAM_PROJECT  |PARTITIONED|
+                  exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$52, $$p] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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_cbo/column/filter/subplan/subplan.032.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/subplan/subplan.032.plan
index b691147..e73a0ff 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/subplan/subplan.032.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/subplan/subplan.032.plan
@@ -1,78 +1,68 @@
-distribute result [$$70] [cardinality: 2.1, op-cost: 0.0, total-cost: 15.41]
+distribute result [$$70] [cardinality: 2.1, op-cost: 0.0, total-cost: 20.41]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
-  exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 15.41]
+  exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 20.41]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    aggregate [$$70] <- [agg-sql-sum($$76)] [cardinality: 2.1, op-cost: 0.0, total-cost: 15.41]
+    aggregate [$$70] <- [agg-sql-sum($$76)] [cardinality: 2.1, op-cost: 0.0, total-cost: 20.41]
     -- AGGREGATE  |UNPARTITIONED|
-      exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 15.41]
+      exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 20.41]
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        aggregate [$$76] <- [agg-sql-count(1)] [cardinality: 2.1, op-cost: 0.0, total-cost: 15.41]
+        aggregate [$$76] <- [agg-sql-count(1)] [cardinality: 2.1, op-cost: 0.0, total-cost: 20.41]
         -- AGGREGATE  |PARTITIONED|
-          exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 15.41]
+          exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 20.41]
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            join (true) [cardinality: 2.1, op-cost: 4.41, total-cost: 15.41]
+            join (true) [cardinality: 2.1, op-cost: 4.41, total-cost: 20.41]
             -- NESTED_LOOP  |PARTITIONED|
               exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([]) [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  select (eq($$p.getField("arrayOrObject").getField("text"), "7")) [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
-                  -- STREAM_SELECT  |PARTITIONED|
-                    project ([$$p]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$p] <- [$$b2] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                      -- ASSIGN  |PARTITIONED|
-                        exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+                select (eq($$p.getField("arrayOrObject").getField("text"), "7")) project: [] [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
+                -- STREAM_SELECT  |PARTITIONED|
+                  assign [$$p] <- [$$b2] project: [$$p] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                  -- ASSIGN  |PARTITIONED|
+                    exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      replicate [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                      -- REPLICATE  |PARTITIONED|
+                        exchange [cardinality: 6.0, op-cost: 4.0, total-cost: 10.0]
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          replicate [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                          -- REPLICATE  |PARTITIONED|
-                            exchange [cardinality: 1.0, op-cost: 4.0, total-cost: 5.0]
+                          project ([$$b2]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            exchange [cardinality: 6.0, op-cost: 4.0, total-cost: 10.0]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              project ([$$b2]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                exchange [cardinality: 1.0, op-cost: 4.0, total-cost: 5.0]
+                              data-scan []<-[$$68, $$b2] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) filter on: or(eq($$b2.getField("arrayOrObject").getField("text"), "7"), eq(scan-collection($$b2.getField("arrayOrObject")).getField("text"), "1")) range-filter on: or(eq($$b2.getField("arrayOrObject").getField("text"), "7"), eq(scan-collection($$b2.getField("arrayOrObject")).getField("text"), "1")) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.0]
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  data-scan []<-[$$68, $$b2] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) filter on: or(eq($$b2.getField("arrayOrObject").getField("text"), "7"), eq(scan-collection($$b2.getField("arrayOrObject")).getField("text"), "1")) range-filter on: or(eq($$b2.getField("arrayOrObject").getField("text"), "7"), eq(scan-collection($$b2.getField("arrayOrObject")).getField("text"), "1")) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.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: 1.0, op-cost: 4.0, total-cost: 5.0]
+                                  empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              exchange [cardinality: 1.0, op-cost: 4.0, total-cost: 10.0]
               -- BROADCAST_EXCHANGE  |PARTITIONED|
-                project ([]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                -- STREAM_PROJECT  |UNPARTITIONED|
-                  select (neq($$69, 0)) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                  -- STREAM_SELECT  |UNPARTITIONED|
-                    aggregate [$$69] <- [agg-sum($$75)] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                    -- AGGREGATE  |UNPARTITIONED|
-                      exchange [cardinality: 1.0, op-cost: 4.0, total-cost: 5.0]
-                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                        aggregate [$$75] <- [agg-count(1)] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                        -- AGGREGATE  |PARTITIONED|
-                          select (eq($$ao.getField("text"), "1")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- STREAM_SELECT  |PARTITIONED|
-                            project ([$$ao]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              unnest $$ao <- scan-collection($$72) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                              -- UNNEST  |PARTITIONED|
-                                project ([$$72]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  assign [$$72] <- [$$b2.getField("arrayOrObject")] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                                  -- ASSIGN  |PARTITIONED|
-                                    exchange [cardinality: 1.0, op-cost: 4.0, total-cost: 5.0]
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      replicate [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                                      -- REPLICATE  |PARTITIONED|
-                                        exchange [cardinality: 1.0, op-cost: 4.0, total-cost: 5.0]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          project ([$$b2]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            exchange [cardinality: 1.0, op-cost: 4.0, total-cost: 5.0]
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              data-scan []<-[$$68, $$b2] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) filter on: or(eq($$b2.getField("arrayOrObject").getField("text"), "7"), eq(scan-collection($$b2.getField("arrayOrObject")).getField("text"), "1")) range-filter on: or(eq($$b2.getField("arrayOrObject").getField("text"), "7"), eq(scan-collection($$b2.getField("arrayOrObject")).getField("text"), "1")) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.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|
+                select (neq($$69, 0)) project: [] [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
+                -- STREAM_SELECT  |UNPARTITIONED|
+                  aggregate [$$69] <- [agg-sum($$75)] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                  -- AGGREGATE  |UNPARTITIONED|
+                    exchange [cardinality: 6.0, op-cost: 4.0, total-cost: 10.0]
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      aggregate [$$75] <- [agg-count(1)] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                      -- AGGREGATE  |PARTITIONED|
+                        select (eq($$ao.getField("text"), "1")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- STREAM_SELECT  |PARTITIONED|
+                          unnest $$ao <- scan-collection($$72) project: [$$ao] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                          -- UNNEST  |PARTITIONED|
+                            assign [$$72] <- [$$b2.getField("arrayOrObject")] project: [$$72] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                            -- ASSIGN  |PARTITIONED|
+                              exchange [cardinality: 6.0, op-cost: 4.0, total-cost: 10.0]
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                replicate [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                                -- REPLICATE  |PARTITIONED|
+                                  exchange [cardinality: 6.0, op-cost: 4.0, total-cost: 10.0]
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    project ([$$b2]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      exchange [cardinality: 6.0, op-cost: 4.0, total-cost: 10.0]
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        data-scan []<-[$$68, $$b2] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) filter on: or(eq($$b2.getField("arrayOrObject").getField("text"), "7"), eq(scan-collection($$b2.getField("arrayOrObject")).getField("text"), "1")) range-filter on: or(eq($$b2.getField("arrayOrObject").getField("text"), "7"), eq(scan-collection($$b2.getField("arrayOrObject")).getField("text"), "1")) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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_cbo/column/filter/subplan/subplan.042.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/subplan/subplan.042.plan
index 7fe9bcc..ccdbba0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/subplan/subplan.042.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/subplan/subplan.042.plan
@@ -14,30 +14,26 @@
             -- NESTED_LOOP  |PARTITIONED|
               exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$71]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$71] <- [$$p.getField("arrayOrObject").getField("text")] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                assign [$$71] <- [$$p.getField("arrayOrObject").getField("text")] project: [$$71] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                -- ASSIGN  |PARTITIONED|
+                  assign [$$p] <- [$$b2] project: [$$p] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
                   -- ASSIGN  |PARTITIONED|
-                    project ([$$p]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$p] <- [$$b2] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                      -- ASSIGN  |PARTITIONED|
-                        exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                    exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      replicate [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                      -- REPLICATE  |PARTITIONED|
+                        exchange [cardinality: 6.0, op-cost: 4.0, total-cost: 10.0]
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          replicate [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                          -- REPLICATE  |PARTITIONED|
+                          project ([$$b2]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                          -- STREAM_PROJECT  |PARTITIONED|
                             exchange [cardinality: 6.0, op-cost: 4.0, total-cost: 10.0]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              project ([$$b2]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                exchange [cardinality: 6.0, op-cost: 4.0, total-cost: 10.0]
+                              data-scan []<-[$$68, $$b2] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.0]
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  data-scan []<-[$$68, $$b2] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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|
+                                  empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
               exchange [cardinality: 1.0, op-cost: 4.0, total-cost: 10.0]
               -- BROADCAST_EXCHANGE  |PARTITIONED|
                 aggregate [$$69] <- [agg-sum($$75)] [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
@@ -48,27 +44,23 @@
                     -- AGGREGATE  |PARTITIONED|
                       select (eq($$ao.getField("text"), "1")) [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
                       -- STREAM_SELECT  |PARTITIONED|
-                        project ([$$ao]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          unnest $$ao <- scan-collection($$72) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                          -- UNNEST  |PARTITIONED|
-                            project ([$$72]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              assign [$$72] <- [$$b2.getField("arrayOrObject")] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                              -- ASSIGN  |PARTITIONED|
+                        unnest $$ao <- scan-collection($$72) project: [$$ao] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                        -- UNNEST  |PARTITIONED|
+                          assign [$$72] <- [$$b2.getField("arrayOrObject")] project: [$$72] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                          -- ASSIGN  |PARTITIONED|
+                            exchange [cardinality: 6.0, op-cost: 4.0, total-cost: 10.0]
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              replicate [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                              -- REPLICATE  |PARTITIONED|
                                 exchange [cardinality: 6.0, op-cost: 4.0, total-cost: 10.0]
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  replicate [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                                  -- REPLICATE  |PARTITIONED|
+                                  project ([$$b2]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     exchange [cardinality: 6.0, op-cost: 4.0, total-cost: 10.0]
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      project ([$$b2]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        exchange [cardinality: 6.0, op-cost: 4.0, total-cost: 10.0]
+                                      data-scan []<-[$$68, $$b2] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.0]
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          data-scan []<-[$$68, $$b2] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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|
+                                          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_cbo/column/filter/subplan/subplan.052.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/subplan/subplan.052.plan
index 5840b49..48c5799 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/subplan/subplan.052.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/subplan/subplan.052.plan
@@ -23,17 +23,15 @@
                               -- NESTED_TUPLE_SOURCE  |LOCAL|
                    } [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
             -- SUBPLAN  |PARTITIONED|
-              project ([$$68]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                assign [$$68] <- [$$p.getField("arrayOrObject")] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                -- ASSIGN  |PARTITIONED|
-                  project ([$$p]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      data-scan []<-[$$66, $$p] <- test.ColumnDataset project ({arrayOrObject:[{text:any}]}) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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|
+              assign [$$68] <- [$$p.getField("arrayOrObject")] project: [$$68] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+              -- ASSIGN  |PARTITIONED|
+                project ([$$p]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                -- STREAM_PROJECT  |PARTITIONED|
+                  exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$66, $$p] <- test.ColumnDataset project ({arrayOrObject:[{text:any}]}) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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_cbo/column/filter/subplan/subplan.062.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/subplan/subplan.062.plan
index 70f874e..a745248 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/subplan/subplan.062.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/filter/subplan/subplan.062.plan
@@ -6,30 +6,26 @@
     -- NESTED_LOOP  |PARTITIONED|
       exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        project ([]) [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          select (eq($$p.getField("arrayOrObject").getField("text"), "7")) [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
-          -- STREAM_SELECT  |PARTITIONED|
-            project ([$$p]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              assign [$$p] <- [$$p2] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-              -- ASSIGN  |PARTITIONED|
-                exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+        select (eq($$p.getField("arrayOrObject").getField("text"), "7")) project: [] [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
+        -- STREAM_SELECT  |PARTITIONED|
+          assign [$$p] <- [$$p2] project: [$$p] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+          -- ASSIGN  |PARTITIONED|
+            exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              replicate [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+              -- REPLICATE  |PARTITIONED|
+                exchange [cardinality: 6.0, op-cost: 4.0, total-cost: 10.0]
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  replicate [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                  -- REPLICATE  |PARTITIONED|
+                  project ([$$p2]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                  -- STREAM_PROJECT  |PARTITIONED|
                     exchange [cardinality: 6.0, op-cost: 4.0, total-cost: 10.0]
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      project ([$$p2]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        exchange [cardinality: 6.0, op-cost: 4.0, total-cost: 10.0]
+                      data-scan []<-[$$62, $$p2] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) filter on: or(eq($$p2.getField("arrayOrObject").getField("text"), "7"), and(ge(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "1"), le(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "2"))) range-filter on: or(eq($$p2.getField("arrayOrObject").getField("text"), "7"), and(ge(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "1"), le(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "2"))) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.0]
+                      -- DATASOURCE_SCAN  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$62, $$p2] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) filter on: or(eq($$p2.getField("arrayOrObject").getField("text"), "7"), and(ge(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "1"), le(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "2"))) range-filter on: or(eq($$p2.getField("arrayOrObject").getField("text"), "7"), and(ge(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "1"), le(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "2"))) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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|
+                          empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
       exchange [cardinality: 1.0, op-cost: 4.0, total-cost: 10.0]
       -- BROADCAST_EXCHANGE  |PARTITIONED|
         aggregate [$$63] <- [agg-sql-sum($$67)] [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
@@ -40,31 +36,25 @@
             -- AGGREGATE  |PARTITIONED|
               select (and(ge($$60, "1"), le($$60, "2"))) [cardinality: 1.0, op-cost: 0.0, total-cost: 6.0]
               -- STREAM_SELECT  |PARTITIONED|
-                project ([$$60]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$60] <- [$$ao.getField("text")] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                  -- ASSIGN  |PARTITIONED|
-                    project ([$$ao]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      unnest $$ao <- scan-collection($$65) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                      -- UNNEST  |PARTITIONED|
-                        project ([$$65]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$65] <- [$$p2.getField("arrayOrObject")] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                          -- ASSIGN  |PARTITIONED|
-                            exchange [cardinality: 6.0, op-cost: 4.0, total-cost: 10.0]
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              replicate [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                              -- REPLICATE  |PARTITIONED|
-                                exchange [cardinality: 6.0, op-cost: 4.0, total-cost: 10.0]
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  project ([$$p2]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    exchange [cardinality: 6.0, op-cost: 4.0, total-cost: 10.0]
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      data-scan []<-[$$62, $$p2] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) filter on: or(eq($$p2.getField("arrayOrObject").getField("text"), "7"), and(ge(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "1"), le(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "2"))) range-filter on: or(eq($$p2.getField("arrayOrObject").getField("text"), "7"), and(ge(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "1"), le(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "2"))) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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|
+                assign [$$60] <- [$$ao.getField("text")] project: [$$60] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                -- ASSIGN  |PARTITIONED|
+                  unnest $$ao <- scan-collection($$65) project: [$$ao] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                  -- UNNEST  |PARTITIONED|
+                    assign [$$65] <- [$$p2.getField("arrayOrObject")] project: [$$65] [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                    -- ASSIGN  |PARTITIONED|
+                      exchange [cardinality: 6.0, op-cost: 4.0, total-cost: 10.0]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        replicate [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                        -- REPLICATE  |PARTITIONED|
+                          exchange [cardinality: 6.0, op-cost: 4.0, total-cost: 10.0]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            project ([$$p2]) [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              exchange [cardinality: 6.0, op-cost: 4.0, total-cost: 10.0]
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                data-scan []<-[$$62, $$p2] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) filter on: or(eq($$p2.getField("arrayOrObject").getField("text"), "7"), and(ge(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "1"), le(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "2"))) range-filter on: or(eq($$p2.getField("arrayOrObject").getField("text"), "7"), and(ge(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "1"), le(scan-collection($$p2.getField("arrayOrObject")).getField("text"), "2"))) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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_cbo/column/pushdown/array-access-pushdown/array-access-pushdown.004.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/array-access-pushdown/array-access-pushdown.004.plan
index 9f01ff8..b6cb8d9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/array-access-pushdown/array-access-pushdown.004.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/array-access-pushdown/array-access-pushdown.004.plan
@@ -2,27 +2,23 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$18]) [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$18] <- [{"display_url": $$22}] [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$22]) [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-          -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-            order (ASC, $$20) [cardinality: 2.0, op-cost: 2.0, total-cost: 4.0]
-            -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-              exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$22, $$20]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$22] <- [get-item($$p.getField("entities").getField("urls"), 0).getField("display_url")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                  -- ASSIGN  |PARTITIONED|
-                    exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+    assign [$$18] <- [{"display_url": $$22}] project: [$$18] [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$22]) [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
+        -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
+          order (ASC, $$20) [cardinality: 2.0, op-cost: 2.0, total-cost: 4.0]
+          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+            exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$22] <- [get-item($$p.getField("entities").getField("urls"), 0).getField("display_url")] project: [$$22, $$20] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+              -- ASSIGN  |PARTITIONED|
+                exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$20, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.0]
+                  -- DATASOURCE_SCAN  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      data-scan []<-[$$20, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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|
+                      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_cbo/column/pushdown/array-access-pushdown/array-access-pushdown.006.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/array-access-pushdown/array-access-pushdown.006.plan
index 4c75f25..574b5e4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/array-access-pushdown/array-access-pushdown.006.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/array-access-pushdown/array-access-pushdown.006.plan
@@ -2,33 +2,27 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$22]) [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$22] <- [{"display_url": $$27}] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$27]) [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
-          -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
-            order (ASC, $$25) [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
-            -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
-              exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$27, $$25]) [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$27] <- [array-star($$24).getField("display_url")] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+    assign [$$22] <- [{"display_url": $$27}] project: [$$22] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$27]) [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+        -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
+          order (ASC, $$25) [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+          -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+            exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$27] <- [array-star($$24).getField("display_url")] project: [$$27, $$25] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+              -- ASSIGN  |PARTITIONED|
+                select (not(is-missing($$24))) [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+                -- STREAM_SELECT  |PARTITIONED|
+                  assign [$$24] <- [$$p.getField("entities").getField("urls")] project: [$$25, $$24] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                   -- ASSIGN  |PARTITIONED|
-                    select (not(is-missing($$24))) [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
-                    -- STREAM_SELECT  |PARTITIONED|
-                      project ([$$25, $$24]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$24] <- [$$p.getField("entities").getField("urls")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                        -- ASSIGN  |PARTITIONED|
-                          exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            data-scan []<-[$$25, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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: 2.0, op-cost: 0.0, total-cost: 2.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$25, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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_cbo/column/pushdown/array-access-pushdown/array-access-pushdown.008.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/array-access-pushdown/array-access-pushdown.008.plan
index 95d96c1..ddb4708 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/array-access-pushdown/array-access-pushdown.008.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/array-access-pushdown/array-access-pushdown.008.plan
@@ -1,32 +1,26 @@
-distribute result [$$28] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+distribute result [$$28] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$28]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$28] <- [{"display_url": $$urls.getField("display_url")}] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$urls]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-          -- SORT_MERGE_EXCHANGE [$$30(ASC) ]  |PARTITIONED|
-            order (ASC, $$30) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-            -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
-              exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$urls, $$30]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  unnest $$urls <- scan-collection($$31) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                  -- UNNEST  |PARTITIONED|
-                    project ([$$30, $$31]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$31] <- [$$p.getField("entities").getField("urls")] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                      -- ASSIGN  |PARTITIONED|
-                        exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$30, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.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|
+    assign [$$28] <- [{"display_url": $$urls.getField("display_url")}] project: [$$28] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$urls]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+        -- SORT_MERGE_EXCHANGE [$$30(ASC) ]  |PARTITIONED|
+          order (ASC, $$30) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+          -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
+            exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              unnest $$urls <- scan-collection($$31) project: [$$urls, $$30] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+              -- UNNEST  |PARTITIONED|
+                assign [$$31] <- [$$p.getField("entities").getField("urls")] project: [$$30, $$31] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                -- ASSIGN  |PARTITIONED|
+                  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$30, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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_cbo/column/pushdown/array-access-pushdown/array-access-pushdown.010.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/array-access-pushdown/array-access-pushdown.010.plan
index 0579a38..ad119ad 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/array-access-pushdown/array-access-pushdown.010.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/array-access-pushdown/array-access-pushdown.010.plan
@@ -25,17 +25,15 @@
                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                      } [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
               -- SUBPLAN  |PARTITIONED|
-                project ([$$51]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$51] <- [$$p.getField("entities").getField("urls")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                  -- ASSIGN  |PARTITIONED|
-                    project ([$$p]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        data-scan []<-[$$49, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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|
+                assign [$$51] <- [$$p.getField("entities").getField("urls")] project: [$$51] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$p]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$49, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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_cbo/column/pushdown/array-access-pushdown/array-access-pushdown.012.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/array-access-pushdown/array-access-pushdown.012.plan
index 7355bbe..1f0567b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/array-access-pushdown/array-access-pushdown.012.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/array-access-pushdown/array-access-pushdown.012.plan
@@ -2,27 +2,23 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$19]) [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$19] <- [get-item(get-item(get-item($$25.getField("coordinates"), 0), 0), 0)] [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$25]) [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-          -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
-            order (ASC, $$21) [cardinality: 2.0, op-cost: 2.0, total-cost: 4.0]
-            -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
-              exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$25, $$21]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$25] <- [$$p.getField("place").getField("bounding_box")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                  -- ASSIGN  |PARTITIONED|
-                    exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+    assign [$$19] <- [get-item(get-item(get-item($$25.getField("coordinates"), 0), 0), 0)] project: [$$19] [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$25]) [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
+        -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+          order (ASC, $$21) [cardinality: 2.0, op-cost: 2.0, total-cost: 4.0]
+          -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+            exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$25] <- [$$p.getField("place").getField("bounding_box")] project: [$$25, $$21] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+              -- ASSIGN  |PARTITIONED|
+                exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$21, $$p] <- test.ColumnDataset project ({place:{bounding_box:{coordinates:[[[any]]]}}}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.0]
+                  -- DATASOURCE_SCAN  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      data-scan []<-[$$21, $$p] <- test.ColumnDataset project ({place:{bounding_box:{coordinates:[[[any]]]}}}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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|
+                      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_cbo/column/pushdown/field-access-pushdown/field-access-pushdown.006.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/field-access-pushdown/field-access-pushdown.006.plan
index 20f5102..3d196fc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/field-access-pushdown/field-access-pushdown.006.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/field-access-pushdown/field-access-pushdown.006.plan
@@ -1,40 +1,38 @@
-distribute result [$$30] [cardinality: 8.17, op-cost: 0.0, total-cost: 66.76]
+distribute result [$$30] [cardinality: 7.0, op-cost: 0.0, total-cost: 61.65]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 8.17, op-cost: 0.0, total-cost: 66.76]
+  exchange [cardinality: 7.0, op-cost: 0.0, total-cost: 61.65]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$30]) [cardinality: 8.17, op-cost: 0.0, total-cost: 66.76]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$30] <- [{"p1": $$p1, "id": $$34}] [cardinality: 8.17, op-cost: 0.0, total-cost: 66.76]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 8.17, op-cost: 0.0, total-cost: 66.76]
-        -- SORT_MERGE_EXCHANGE [$$34(ASC) ]  |PARTITIONED|
-          order (ASC, $$34) [cardinality: 8.17, op-cost: 24.76, total-cost: 66.76]
-          -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
-            exchange [cardinality: 8.17, op-cost: 0.0, total-cost: 42.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$p1, $$34]) [cardinality: 8.17, op-cost: 0.0, total-cost: 42.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                exchange [cardinality: 8.17, op-cost: 0.0, total-cost: 42.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  join (eq($$33, $$34)) [cardinality: 8.17, op-cost: 14.0, total-cost: 42.0]
-                  -- HYBRID_HASH_JOIN [$$33][$$34]  |PARTITIONED|
-                    exchange [cardinality: 7.0, op-cost: 7.0, total-cost: 14.0]
-                    -- HASH_PARTITION_EXCHANGE [$$33]  |PARTITIONED|
-                      data-scan []<-[$$33, $$p1] <- test.ColumnDataset1 [cardinality: 7.0, op-cost: 7.0, total-cost: 7.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: 7.0, op-cost: 7.0, total-cost: 14.0]
-                    -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
-                      project ([$$34]) [cardinality: 7.0, op-cost: 0.0, total-cost: 7.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        exchange [cardinality: 7.0, op-cost: 7.0, total-cost: 14.0]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$34, $$p2] <- test.ColumnDataset2 project ({}) [cardinality: 7.0, op-cost: 7.0, total-cost: 7.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|
+    assign [$$30] <- [{"p1": $$p1, "id": $$34}] project: [$$30] [cardinality: 7.0, op-cost: 0.0, total-cost: 61.65]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 7.0, op-cost: 0.0, total-cost: 61.65]
+      -- SORT_MERGE_EXCHANGE [$$34(ASC) ]  |PARTITIONED|
+        order (ASC, $$34) [cardinality: 7.0, op-cost: 19.65, total-cost: 61.65]
+        -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
+          exchange [cardinality: 7.0, op-cost: 0.0, total-cost: 42.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            project ([$$p1, $$34]) [cardinality: 7.0, op-cost: 0.0, total-cost: 42.0]
+            -- STREAM_PROJECT  |PARTITIONED|
+              exchange [cardinality: 7.0, op-cost: 0.0, total-cost: 42.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                join (eq($$33, $$34)) [cardinality: 7.0, op-cost: 14.0, total-cost: 42.0]
+                -- HYBRID_HASH_JOIN [$$33][$$34]  |PARTITIONED|
+                  exchange [cardinality: 7.0, op-cost: 7.0, total-cost: 14.0]
+                  -- HASH_PARTITION_EXCHANGE [$$33]  |PARTITIONED|
+                    data-scan []<-[$$33, $$p1] <- test.ColumnDataset1 [cardinality: 7.0, op-cost: 7.0, total-cost: 7.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: 7.0, op-cost: 7.0, total-cost: 14.0]
+                  -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
+                    project ([$$34]) [cardinality: 7.0, op-cost: 0.0, total-cost: 7.0]
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      exchange [cardinality: 7.0, op-cost: 7.0, total-cost: 14.0]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        data-scan []<-[$$34, $$p2] <- test.ColumnDataset2 project ({}) [cardinality: 7.0, op-cost: 7.0, total-cost: 7.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_cbo/column/pushdown/field-access-pushdown/field-access-pushdown.008.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/field-access-pushdown/field-access-pushdown.008.plan
index 9022614..f7797b0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/field-access-pushdown/field-access-pushdown.008.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/field-access-pushdown/field-access-pushdown.008.plan
@@ -1,50 +1,44 @@
-distribute result [$$31] [cardinality: 9.33, op-cost: 0.0, total-cost: 75.06]
+distribute result [$$31] [cardinality: 8.0, op-cost: 0.0, total-cost: 69.0]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 9.33, op-cost: 0.0, total-cost: 75.06]
+  exchange [cardinality: 8.0, op-cost: 0.0, total-cost: 69.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$31]) [cardinality: 9.33, op-cost: 0.0, total-cost: 75.06]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$31] <- [{"age": $$38, "name": $$39}] [cardinality: 9.33, op-cost: 0.0, total-cost: 75.06]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$38, $$39]) [cardinality: 9.33, op-cost: 0.0, total-cost: 75.06]
-        -- STREAM_PROJECT  |PARTITIONED|
-          exchange [cardinality: 9.33, op-cost: 0.0, total-cost: 75.06]
-          -- SORT_MERGE_EXCHANGE [$$34(ASC) ]  |PARTITIONED|
-            order (ASC, $$34) [cardinality: 9.33, op-cost: 30.06, total-cost: 75.06]
-            -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
-              exchange [cardinality: 9.33, op-cost: 0.0, total-cost: 45.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$38, $$39, $$34]) [cardinality: 9.33, op-cost: 0.0, total-cost: 45.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  exchange [cardinality: 9.33, op-cost: 0.0, total-cost: 45.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    join (eq($$33, $$34)) [cardinality: 9.33, op-cost: 15.0, total-cost: 45.0]
-                    -- HYBRID_HASH_JOIN [$$34][$$33]  |PARTITIONED|
-                      exchange [cardinality: 8.0, op-cost: 8.0, total-cost: 16.0]
-                      -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
-                        project ([$$39, $$34]) [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$39] <- [$$p2.getField("name")] [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
-                          -- ASSIGN  |PARTITIONED|
-                            exchange [cardinality: 8.0, op-cost: 8.0, total-cost: 16.0]
+    assign [$$31] <- [{"age": $$38, "name": $$39}] project: [$$31] [cardinality: 8.0, op-cost: 0.0, total-cost: 69.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$38, $$39]) [cardinality: 8.0, op-cost: 0.0, total-cost: 69.0]
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange [cardinality: 8.0, op-cost: 0.0, total-cost: 69.0]
+        -- SORT_MERGE_EXCHANGE [$$34(ASC) ]  |PARTITIONED|
+          order (ASC, $$34) [cardinality: 8.0, op-cost: 24.0, total-cost: 69.0]
+          -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
+            exchange [cardinality: 8.0, op-cost: 0.0, total-cost: 45.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              project ([$$38, $$39, $$34]) [cardinality: 8.0, op-cost: 0.0, total-cost: 45.0]
+              -- STREAM_PROJECT  |PARTITIONED|
+                exchange [cardinality: 8.0, op-cost: 0.0, total-cost: 45.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  join (eq($$33, $$34)) [cardinality: 8.0, op-cost: 15.0, total-cost: 45.0]
+                  -- HYBRID_HASH_JOIN [$$34][$$33]  |PARTITIONED|
+                    exchange [cardinality: 8.0, op-cost: 8.0, total-cost: 16.0]
+                    -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
+                      assign [$$39] <- [$$p2.getField("name")] project: [$$39, $$34] [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
+                      -- ASSIGN  |PARTITIONED|
+                        exchange [cardinality: 8.0, op-cost: 8.0, total-cost: 16.0]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$34, $$p2] <- test.ColumnDataset3 project ({name:any}) [cardinality: 8.0, op-cost: 8.0, total-cost: 8.0]
+                          -- DATASOURCE_SCAN  |PARTITIONED|
+                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              data-scan []<-[$$34, $$p2] <- test.ColumnDataset3 project ({name:any}) [cardinality: 8.0, op-cost: 8.0, total-cost: 8.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: 7.0, op-cost: 7.0, total-cost: 14.0]
-                      -- HASH_PARTITION_EXCHANGE [$$33]  |PARTITIONED|
-                        project ([$$38, $$33]) [cardinality: 7.0, op-cost: 0.0, total-cost: 7.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$38] <- [$$p1.getField("age")] [cardinality: 7.0, op-cost: 0.0, total-cost: 7.0]
-                          -- ASSIGN  |PARTITIONED|
-                            exchange [cardinality: 7.0, op-cost: 7.0, total-cost: 14.0]
+                              empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    exchange [cardinality: 7.0, op-cost: 7.0, total-cost: 14.0]
+                    -- HASH_PARTITION_EXCHANGE [$$33]  |PARTITIONED|
+                      assign [$$38] <- [$$p1.getField("age")] project: [$$38, $$33] [cardinality: 7.0, op-cost: 0.0, total-cost: 7.0]
+                      -- ASSIGN  |PARTITIONED|
+                        exchange [cardinality: 7.0, op-cost: 7.0, total-cost: 14.0]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$33, $$p1] <- test.ColumnDataset1 project ({age:any}) [cardinality: 7.0, op-cost: 7.0, total-cost: 7.0]
+                          -- DATASOURCE_SCAN  |PARTITIONED|
+                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              data-scan []<-[$$33, $$p1] <- test.ColumnDataset1 project ({age:any}) [cardinality: 7.0, op-cost: 7.0, total-cost: 7.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|
+                              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_cbo/column/pushdown/field-access-pushdown/field-access-pushdown.010.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/field-access-pushdown/field-access-pushdown.010.plan
index 2f7637f..82b6d69 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/field-access-pushdown/field-access-pushdown.010.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/field-access-pushdown/field-access-pushdown.010.plan
@@ -2,25 +2,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$18]) [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$18] <- [{"id": $$20.getField("id"), "name": $$20.getField("name")}] [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
+    assign [$$18] <- [{"id": $$20.getField("id"), "name": $$20.getField("name")}] project: [$$18] [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
+    -- ASSIGN  |PARTITIONED|
+      assign [$$20] <- [$$p.getField("user")] project: [$$20] [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
       -- ASSIGN  |PARTITIONED|
-        project ([$$20]) [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
+        project ([$$p]) [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
         -- STREAM_PROJECT  |PARTITIONED|
-          assign [$$20] <- [$$p.getField("user")] [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-          -- ASSIGN  |PARTITIONED|
-            project ([$$p]) [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-              -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
-                order (ASC, $$21) [cardinality: 2.0, op-cost: 2.0, total-cost: 4.0]
-                -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
-                  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+          exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
+          -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+            order (ASC, $$21) [cardinality: 2.0, op-cost: 2.0, total-cost: 4.0]
+            -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+              exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                data-scan []<-[$$21, $$p] <- test.ColumnDataset4 project ({user:{name:any,id:any}}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.0]
+                -- DATASOURCE_SCAN  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$21, $$p] <- test.ColumnDataset4 project ({user:{name:any,id:any}}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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|
+                    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_cbo/column/pushdown/heterogeneous-access-pushdown/heterogeneous-access-pushdown.004.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/heterogeneous-access-pushdown/heterogeneous-access-pushdown.004.plan
index 76ea941..294dbea 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/heterogeneous-access-pushdown/heterogeneous-access-pushdown.004.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/heterogeneous-access-pushdown/heterogeneous-access-pushdown.004.plan
@@ -2,25 +2,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 21.51]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$21]) [cardinality: 6.0, op-cost: 0.0, total-cost: 21.51]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$21] <- [switch-case(true, is-array($$23), array-star($$23).getField("text"), $$23.getField("text"))] [cardinality: 6.0, op-cost: 0.0, total-cost: 21.51]
+    assign [$$21] <- [switch-case(true, is-array($$23), array-star($$23).getField("text"), $$23.getField("text"))] project: [$$21] [cardinality: 6.0, op-cost: 0.0, total-cost: 21.51]
+    -- ASSIGN  |PARTITIONED|
+      assign [$$23] <- [$$p.getField("arrayOrObject")] project: [$$23] [cardinality: 6.0, op-cost: 0.0, total-cost: 21.51]
       -- ASSIGN  |PARTITIONED|
-        project ([$$23]) [cardinality: 6.0, op-cost: 0.0, total-cost: 21.51]
+        project ([$$p]) [cardinality: 6.0, op-cost: 0.0, total-cost: 21.51]
         -- STREAM_PROJECT  |PARTITIONED|
-          assign [$$23] <- [$$p.getField("arrayOrObject")] [cardinality: 6.0, op-cost: 0.0, total-cost: 21.51]
-          -- ASSIGN  |PARTITIONED|
-            project ([$$p]) [cardinality: 6.0, op-cost: 0.0, total-cost: 21.51]
-            -- STREAM_PROJECT  |PARTITIONED|
-              exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 21.51]
-              -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-                order (ASC, $$24) [cardinality: 6.0, op-cost: 15.51, total-cost: 21.51]
-                -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
-                  exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+          exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 21.51]
+          -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+            order (ASC, $$24) [cardinality: 6.0, op-cost: 15.51, total-cost: 21.51]
+            -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+              exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 6.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                data-scan []<-[$$24, $$p] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.0]
+                -- DATASOURCE_SCAN  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$24, $$p] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) [cardinality: 6.0, op-cost: 6.0, total-cost: 6.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|
+                    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_cbo/column/pushdown/join/condition-pushdown/condition-pushdown.012.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/join/condition-pushdown/condition-pushdown.012.plan
index 76ba916..f087fe5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/join/condition-pushdown/condition-pushdown.012.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/join/condition-pushdown/condition-pushdown.012.plan
@@ -2,53 +2,47 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$32]) [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$32] <- [{"c1": $$40, "c2": $$41}] [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
-        -- SORT_MERGE_EXCHANGE [$$40(ASC), $$41(ASC) ]  |PARTITIONED|
-          order (ASC, $$40) (ASC, $$41) [cardinality: 2.1, op-cost: 2.25, total-cost: 13.35]
-          -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
-            exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 11.1]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$40, $$41]) [cardinality: 2.1, op-cost: 0.0, total-cost: 11.1]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select (eq($$37, $$t2.getField("c_s"))) [cardinality: 2.1, op-cost: 6.1, total-cost: 11.1]
-                -- STREAM_SELECT  |PARTITIONED|
-                  assign [$$41] <- [$$t2.getField("c_x")] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                  -- ASSIGN  |PARTITIONED|
-                    project ([$$40, $$37, $$t2]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+    assign [$$32] <- [{"c1": $$40, "c2": $$41}] project: [$$32] [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
+      -- SORT_MERGE_EXCHANGE [$$40(ASC), $$41(ASC) ]  |PARTITIONED|
+        order (ASC, $$40) (ASC, $$41) [cardinality: 2.1, op-cost: 2.25, total-cost: 13.35]
+        -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+          exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 11.1]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select (eq($$37, $$t2.getField("c_s"))) project: [$$40, $$41] [cardinality: 2.1, op-cost: 6.1, total-cost: 11.1]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$41] <- [$$t2.getField("c_x")] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+              -- ASSIGN  |PARTITIONED|
+                project ([$$40, $$37, $$t2]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+                -- STREAM_PROJECT  |PARTITIONED|
+                  exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    unnest-map [$$36, $$t2] <- index-search("TestOpenColumn2", 0, "Default", "test", "TestOpenColumn2", true, false, 1, $$45, 1, $$45, true, true, true) project ({c_s:any,c_x:any}) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.0]
+                    -- BTREE_SEARCH  |PARTITIONED|
+                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        unnest-map [$$36, $$t2] <- index-search("TestOpenColumn2", 0, "Default", "test", "TestOpenColumn2", true, false, 1, $$45, 1, $$45, true, true, true) project ({c_s:any,c_x:any}) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.0]
-                        -- BTREE_SEARCH  |PARTITIONED|
+                        order (ASC, $$45) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
                           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            order (ASC, $$45) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
+                            project ([$$40, $$37, $$45]) [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|
-                                project ([$$40, $$37, $$45]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_PROJECT  |PARTITIONED|
+                                unnest-map [$$44, $$45] <- index-search("idx_column_t2_s", 0, "Default", "test", "TestOpenColumn2", true, true, 1, $$37, 1, $$37, true, true, true) [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|
-                                    unnest-map [$$44, $$45] <- index-search("idx_column_t2_s", 0, "Default", "test", "TestOpenColumn2", true, true, 1, $$37, 1, $$37, true, true, true) [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]
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        project ([$$37, $$40]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          assign [$$37, $$40] <- [to-string($$t1.getField("c_s")), $$t1.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- ASSIGN  |PARTITIONED|
-                                            project ([$$t1]) [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 []<-[$$35, $$t1] <- test.TestOpenColumn1 project ({c_s:any,c_x:any}) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.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|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    assign [$$37, $$40] <- [to-string($$t1.getField("c_s")), $$t1.getField("c_x")] project: [$$37, $$40] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- ASSIGN  |PARTITIONED|
+                                      project ([$$t1]) [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 []<-[$$35, $$t1] <- test.TestOpenColumn1 project ({c_s:any,c_x:any}) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.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_cbo/column/pushdown/join/condition-pushdown/condition-pushdown.014.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/join/condition-pushdown/condition-pushdown.014.plan
index e3ab5d1..e475f9b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/join/condition-pushdown/condition-pushdown.014.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/join/condition-pushdown/condition-pushdown.014.plan
@@ -2,59 +2,53 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$37]) [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$37] <- [{"c1": $$47, "c2": $$48}] [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
-        -- SORT_MERGE_EXCHANGE [$$47(ASC), $$48(ASC) ]  |PARTITIONED|
-          order (ASC, $$47) (ASC, $$48) [cardinality: 2.1, op-cost: 2.25, total-cost: 13.35]
-          -- STABLE_SORT [$$47(ASC), $$48(ASC)]  |PARTITIONED|
-            exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 11.1]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$47, $$48]) [cardinality: 2.1, op-cost: 0.0, total-cost: 11.1]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select (eq($$42, $$t2.getField("c_s"))) [cardinality: 2.1, op-cost: 6.1, total-cost: 11.1]
-                -- STREAM_SELECT  |PARTITIONED|
-                  select (eq($$t2.getField("c_i64"), 2.25)) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                  -- STREAM_SELECT  |PARTITIONED|
-                    assign [$$48] <- [$$t2.getField("c_x")] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                    -- ASSIGN  |PARTITIONED|
-                      project ([$$47, $$42, $$t2]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+    assign [$$37] <- [{"c1": $$47, "c2": $$48}] project: [$$37] [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
+      -- SORT_MERGE_EXCHANGE [$$47(ASC), $$48(ASC) ]  |PARTITIONED|
+        order (ASC, $$47) (ASC, $$48) [cardinality: 2.1, op-cost: 2.25, total-cost: 13.35]
+        -- STABLE_SORT [$$47(ASC), $$48(ASC)]  |PARTITIONED|
+          exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 11.1]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select (eq($$42, $$t2.getField("c_s"))) project: [$$47, $$48] [cardinality: 2.1, op-cost: 6.1, total-cost: 11.1]
+            -- STREAM_SELECT  |PARTITIONED|
+              select (eq($$t2.getField("c_i64"), 2.25)) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                assign [$$48] <- [$$t2.getField("c_x")] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$47, $$42, $$t2]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      unnest-map [$$41, $$t2] <- index-search("TestOpenColumn2", 0, "Default", "test", "TestOpenColumn2", true, false, 1, $$52, 1, $$52, true, true, true) project ({c_i64:any,c_s:any,c_x:any}) range-filter on: eq($$t2.getField("c_i64"), 2.25) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.0]
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          unnest-map [$$41, $$t2] <- index-search("TestOpenColumn2", 0, "Default", "test", "TestOpenColumn2", true, false, 1, $$52, 1, $$52, true, true, true) project ({c_i64:any,c_s:any,c_x:any}) range-filter on: eq($$t2.getField("c_i64"), 2.25) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.0]
-                          -- BTREE_SEARCH  |PARTITIONED|
+                          order (ASC, $$52) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
                             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              order (ASC, $$52) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
+                              project ([$$47, $$42, $$52]) [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|
-                                  project ([$$47, $$42, $$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  unnest-map [$$51, $$52] <- index-search("idx_column_t2_s", 0, "Default", "test", "TestOpenColumn2", true, true, 1, $$42, 1, $$42, true, true, true) [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|
-                                      unnest-map [$$51, $$52] <- index-search("idx_column_t2_s", 0, "Default", "test", "TestOpenColumn2", true, true, 1, $$42, 1, $$42, true, true, true) [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]
-                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                          project ([$$47, $$42]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            assign [$$42] <- [to-string($$t1.getField("c_s"))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- ASSIGN  |PARTITIONED|
-                                              select (eq($$t1.getField("c_i64"), 2)) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                assign [$$47] <- [$$t1.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- ASSIGN  |PARTITIONED|
-                                                  project ([$$t1]) [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 []<-[$$40, $$t1] <- test.TestOpenColumn1 project ({c_i64:any,c_s:any,c_x:any}) filter on: eq($$t1.getField("c_i64"), 2) range-filter on: eq($$t1.getField("c_i64"), 2) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.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|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      assign [$$42] <- [to-string($$t1.getField("c_s"))] project: [$$47, $$42] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- ASSIGN  |PARTITIONED|
+                                        select (eq($$t1.getField("c_i64"), 2)) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          assign [$$47] <- [$$t1.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- ASSIGN  |PARTITIONED|
+                                            project ([$$t1]) [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 []<-[$$40, $$t1] <- test.TestOpenColumn1 project ({c_i64:any,c_s:any,c_x:any}) filter on: eq($$t1.getField("c_i64"), 2) range-filter on: eq($$t1.getField("c_i64"), 2) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.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_cbo/column/pushdown/join/condition-pushdown/condition-pushdown.021.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/join/condition-pushdown/condition-pushdown.021.plan
index 9f54990..492006e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/join/condition-pushdown/condition-pushdown.021.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/join/condition-pushdown/condition-pushdown.021.plan
@@ -2,53 +2,47 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$32]) [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$32] <- [{"c1": $$40, "c2": $$41}] [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
-        -- SORT_MERGE_EXCHANGE [$$40(ASC), $$41(ASC) ]  |PARTITIONED|
-          order (ASC, $$40) (ASC, $$41) [cardinality: 2.1, op-cost: 2.25, total-cost: 13.35]
-          -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
-            exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 11.1]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$40, $$41]) [cardinality: 2.1, op-cost: 0.0, total-cost: 11.1]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select (eq($$37, $$t2.getField("c_s"))) [cardinality: 2.1, op-cost: 6.1, total-cost: 11.1]
-                -- STREAM_SELECT  |PARTITIONED|
-                  assign [$$41] <- [$$t2.getField("c_x")] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                  -- ASSIGN  |PARTITIONED|
-                    project ([$$40, $$37, $$t2]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+    assign [$$32] <- [{"c1": $$40, "c2": $$41}] project: [$$32] [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
+      -- SORT_MERGE_EXCHANGE [$$40(ASC), $$41(ASC) ]  |PARTITIONED|
+        order (ASC, $$40) (ASC, $$41) [cardinality: 2.1, op-cost: 2.25, total-cost: 13.35]
+        -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+          exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 11.1]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select (eq($$37, $$t2.getField("c_s"))) project: [$$40, $$41] [cardinality: 2.1, op-cost: 6.1, total-cost: 11.1]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$41] <- [$$t2.getField("c_x")] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+              -- ASSIGN  |PARTITIONED|
+                project ([$$40, $$37, $$t2]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+                -- STREAM_PROJECT  |PARTITIONED|
+                  exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    unnest-map [$$36, $$t2] <- index-search("TestOpenRow2", 0, "Default", "test", "TestOpenRow2", true, false, 1, $$45, 1, $$45, true, true, true) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.0]
+                    -- BTREE_SEARCH  |PARTITIONED|
+                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        unnest-map [$$36, $$t2] <- index-search("TestOpenRow2", 0, "Default", "test", "TestOpenRow2", true, false, 1, $$45, 1, $$45, true, true, true) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.0]
-                        -- BTREE_SEARCH  |PARTITIONED|
+                        order (ASC, $$45) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
                           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            order (ASC, $$45) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
+                            project ([$$40, $$37, $$45]) [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|
-                                project ([$$40, $$37, $$45]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_PROJECT  |PARTITIONED|
+                                unnest-map [$$44, $$45] <- index-search("idx_row_t2_s", 0, "Default", "test", "TestOpenRow2", true, true, 1, $$37, 1, $$37, true, true, true) [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|
-                                    unnest-map [$$44, $$45] <- index-search("idx_row_t2_s", 0, "Default", "test", "TestOpenRow2", true, true, 1, $$37, 1, $$37, true, true, true) [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]
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        project ([$$37, $$40]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          assign [$$37, $$40] <- [to-string($$t1.getField("c_s")), $$t1.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- ASSIGN  |PARTITIONED|
-                                            project ([$$t1]) [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 []<-[$$35, $$t1] <- test.TestOpenColumn1 project ({c_s:any,c_x:any}) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.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|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    assign [$$37, $$40] <- [to-string($$t1.getField("c_s")), $$t1.getField("c_x")] project: [$$37, $$40] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- ASSIGN  |PARTITIONED|
+                                      project ([$$t1]) [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 []<-[$$35, $$t1] <- test.TestOpenColumn1 project ({c_s:any,c_x:any}) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.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_cbo/column/pushdown/join/condition-pushdown/condition-pushdown.023.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/join/condition-pushdown/condition-pushdown.023.plan
index f9b8fa0..81041d1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/join/condition-pushdown/condition-pushdown.023.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/join/condition-pushdown/condition-pushdown.023.plan
@@ -2,59 +2,53 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$37]) [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$37] <- [{"c1": $$47, "c2": $$48}] [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
-        -- SORT_MERGE_EXCHANGE [$$47(ASC), $$48(ASC) ]  |PARTITIONED|
-          order (ASC, $$47) (ASC, $$48) [cardinality: 2.1, op-cost: 2.25, total-cost: 13.35]
-          -- STABLE_SORT [$$47(ASC), $$48(ASC)]  |PARTITIONED|
-            exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 11.1]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$47, $$48]) [cardinality: 2.1, op-cost: 0.0, total-cost: 11.1]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select (eq($$42, $$t2.getField("c_s"))) [cardinality: 2.1, op-cost: 6.1, total-cost: 11.1]
-                -- STREAM_SELECT  |PARTITIONED|
-                  select (eq($$t2.getField("c_i64"), 2.25)) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                  -- STREAM_SELECT  |PARTITIONED|
-                    assign [$$48] <- [$$t2.getField("c_x")] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                    -- ASSIGN  |PARTITIONED|
-                      project ([$$47, $$42, $$t2]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+    assign [$$37] <- [{"c1": $$47, "c2": $$48}] project: [$$37] [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
+      -- SORT_MERGE_EXCHANGE [$$47(ASC), $$48(ASC) ]  |PARTITIONED|
+        order (ASC, $$47) (ASC, $$48) [cardinality: 2.1, op-cost: 2.25, total-cost: 13.35]
+        -- STABLE_SORT [$$47(ASC), $$48(ASC)]  |PARTITIONED|
+          exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 11.1]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select (eq($$42, $$t2.getField("c_s"))) project: [$$47, $$48] [cardinality: 2.1, op-cost: 6.1, total-cost: 11.1]
+            -- STREAM_SELECT  |PARTITIONED|
+              select (eq($$t2.getField("c_i64"), 2.25)) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                assign [$$48] <- [$$t2.getField("c_x")] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$47, $$42, $$t2]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      unnest-map [$$41, $$t2] <- index-search("TestOpenRow2", 0, "Default", "test", "TestOpenRow2", true, false, 1, $$52, 1, $$52, true, true, true) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.0]
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          unnest-map [$$41, $$t2] <- index-search("TestOpenRow2", 0, "Default", "test", "TestOpenRow2", true, false, 1, $$52, 1, $$52, true, true, true) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.0]
-                          -- BTREE_SEARCH  |PARTITIONED|
+                          order (ASC, $$52) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
                             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              order (ASC, $$52) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
+                              project ([$$47, $$42, $$52]) [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|
-                                  project ([$$47, $$42, $$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  unnest-map [$$51, $$52] <- index-search("idx_row_t2_s", 0, "Default", "test", "TestOpenRow2", true, true, 1, $$42, 1, $$42, true, true, true) [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|
-                                      unnest-map [$$51, $$52] <- index-search("idx_row_t2_s", 0, "Default", "test", "TestOpenRow2", true, true, 1, $$42, 1, $$42, true, true, true) [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]
-                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                          project ([$$47, $$42]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            assign [$$42] <- [to-string($$t1.getField("c_s"))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- ASSIGN  |PARTITIONED|
-                                              select (eq($$t1.getField("c_i64"), 2)) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                assign [$$47] <- [$$t1.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- ASSIGN  |PARTITIONED|
-                                                  project ([$$t1]) [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 []<-[$$40, $$t1] <- test.TestOpenColumn1 project ({c_i64:any,c_s:any,c_x:any}) filter on: eq($$t1.getField("c_i64"), 2) range-filter on: eq($$t1.getField("c_i64"), 2) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.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|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      assign [$$42] <- [to-string($$t1.getField("c_s"))] project: [$$47, $$42] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- ASSIGN  |PARTITIONED|
+                                        select (eq($$t1.getField("c_i64"), 2)) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          assign [$$47] <- [$$t1.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- ASSIGN  |PARTITIONED|
+                                            project ([$$t1]) [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 []<-[$$40, $$t1] <- test.TestOpenColumn1 project ({c_i64:any,c_s:any,c_x:any}) filter on: eq($$t1.getField("c_i64"), 2) range-filter on: eq($$t1.getField("c_i64"), 2) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.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_cbo/column/pushdown/join/condition-pushdown/condition-pushdown.031.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/join/condition-pushdown/condition-pushdown.031.plan
index 446cab5..65e0d41 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/join/condition-pushdown/condition-pushdown.031.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/join/condition-pushdown/condition-pushdown.031.plan
@@ -2,53 +2,47 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$32]) [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$32] <- [{"c1": $$40, "c2": $$41}] [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
-        -- SORT_MERGE_EXCHANGE [$$40(ASC), $$41(ASC) ]  |PARTITIONED|
-          order (ASC, $$40) (ASC, $$41) [cardinality: 2.1, op-cost: 2.25, total-cost: 13.35]
-          -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
-            exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 11.1]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$40, $$41]) [cardinality: 2.1, op-cost: 0.0, total-cost: 11.1]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select (eq($$37, $$t2.getField("c_s"))) [cardinality: 2.1, op-cost: 6.1, total-cost: 11.1]
-                -- STREAM_SELECT  |PARTITIONED|
-                  assign [$$41] <- [$$t2.getField("c_x")] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                  -- ASSIGN  |PARTITIONED|
-                    project ([$$40, $$37, $$t2]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+    assign [$$32] <- [{"c1": $$40, "c2": $$41}] project: [$$32] [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
+      -- SORT_MERGE_EXCHANGE [$$40(ASC), $$41(ASC) ]  |PARTITIONED|
+        order (ASC, $$40) (ASC, $$41) [cardinality: 2.1, op-cost: 2.25, total-cost: 13.35]
+        -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+          exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 11.1]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select (eq($$37, $$t2.getField("c_s"))) project: [$$40, $$41] [cardinality: 2.1, op-cost: 6.1, total-cost: 11.1]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$41] <- [$$t2.getField("c_x")] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+              -- ASSIGN  |PARTITIONED|
+                project ([$$40, $$37, $$t2]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+                -- STREAM_PROJECT  |PARTITIONED|
+                  exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    unnest-map [$$36, $$t2] <- index-search("TestOpenColumn2", 0, "Default", "test", "TestOpenColumn2", true, false, 1, $$45, 1, $$45, true, true, true) project ({c_s:any,c_x:any}) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.0]
+                    -- BTREE_SEARCH  |PARTITIONED|
+                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        unnest-map [$$36, $$t2] <- index-search("TestOpenColumn2", 0, "Default", "test", "TestOpenColumn2", true, false, 1, $$45, 1, $$45, true, true, true) project ({c_s:any,c_x:any}) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.0]
-                        -- BTREE_SEARCH  |PARTITIONED|
+                        order (ASC, $$45) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
                           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            order (ASC, $$45) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
+                            project ([$$40, $$37, $$45]) [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|
-                                project ([$$40, $$37, $$45]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_PROJECT  |PARTITIONED|
+                                unnest-map [$$44, $$45] <- index-search("idx_column_t2_s", 0, "Default", "test", "TestOpenColumn2", true, true, 1, $$37, 1, $$37, true, true, true) [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|
-                                    unnest-map [$$44, $$45] <- index-search("idx_column_t2_s", 0, "Default", "test", "TestOpenColumn2", true, true, 1, $$37, 1, $$37, true, true, true) [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]
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        project ([$$37, $$40]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          assign [$$37, $$40] <- [to-string($$t1.getField("c_s")), $$t1.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- ASSIGN  |PARTITIONED|
-                                            project ([$$t1]) [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 []<-[$$35, $$t1] <- test.TestOpenRow1 [cardinality: 1.0, op-cost: 1.0, total-cost: 1.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|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    assign [$$37, $$40] <- [to-string($$t1.getField("c_s")), $$t1.getField("c_x")] project: [$$37, $$40] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- ASSIGN  |PARTITIONED|
+                                      project ([$$t1]) [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 []<-[$$35, $$t1] <- test.TestOpenRow1 [cardinality: 1.0, op-cost: 1.0, total-cost: 1.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_cbo/column/pushdown/join/condition-pushdown/condition-pushdown.033.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/join/condition-pushdown/condition-pushdown.033.plan
index dce1f3f..621471e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/join/condition-pushdown/condition-pushdown.033.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/join/condition-pushdown/condition-pushdown.033.plan
@@ -2,59 +2,53 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$37]) [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$37] <- [{"c1": $$47, "c2": $$48}] [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
-        -- SORT_MERGE_EXCHANGE [$$47(ASC), $$48(ASC) ]  |PARTITIONED|
-          order (ASC, $$47) (ASC, $$48) [cardinality: 2.1, op-cost: 2.25, total-cost: 13.35]
-          -- STABLE_SORT [$$47(ASC), $$48(ASC)]  |PARTITIONED|
-            exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 11.1]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$47, $$48]) [cardinality: 2.1, op-cost: 0.0, total-cost: 11.1]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select (eq($$42, $$t2.getField("c_s"))) [cardinality: 2.1, op-cost: 6.1, total-cost: 11.1]
-                -- STREAM_SELECT  |PARTITIONED|
-                  select (eq($$t2.getField("c_i64"), 2.25)) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                  -- STREAM_SELECT  |PARTITIONED|
-                    assign [$$48] <- [$$t2.getField("c_x")] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                    -- ASSIGN  |PARTITIONED|
-                      project ([$$47, $$42, $$t2]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+    assign [$$37] <- [{"c1": $$47, "c2": $$48}] project: [$$37] [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 13.35]
+      -- SORT_MERGE_EXCHANGE [$$47(ASC), $$48(ASC) ]  |PARTITIONED|
+        order (ASC, $$47) (ASC, $$48) [cardinality: 2.1, op-cost: 2.25, total-cost: 13.35]
+        -- STABLE_SORT [$$47(ASC), $$48(ASC)]  |PARTITIONED|
+          exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 11.1]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select (eq($$42, $$t2.getField("c_s"))) project: [$$47, $$48] [cardinality: 2.1, op-cost: 6.1, total-cost: 11.1]
+            -- STREAM_SELECT  |PARTITIONED|
+              select (eq($$t2.getField("c_i64"), 2.25)) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                assign [$$48] <- [$$t2.getField("c_x")] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$47, $$42, $$t2]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      unnest-map [$$41, $$t2] <- index-search("TestOpenColumn2", 0, "Default", "test", "TestOpenColumn2", true, false, 1, $$52, 1, $$52, true, true, true) project ({c_i64:any,c_s:any,c_x:any}) range-filter on: eq($$t2.getField("c_i64"), 2.25) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.0]
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          unnest-map [$$41, $$t2] <- index-search("TestOpenColumn2", 0, "Default", "test", "TestOpenColumn2", true, false, 1, $$52, 1, $$52, true, true, true) project ({c_i64:any,c_s:any,c_x:any}) range-filter on: eq($$t2.getField("c_i64"), 2.25) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.0]
-                          -- BTREE_SEARCH  |PARTITIONED|
+                          order (ASC, $$52) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
                             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              order (ASC, $$52) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
+                              project ([$$47, $$42, $$52]) [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|
-                                  project ([$$47, $$42, $$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                  unnest-map [$$51, $$52] <- index-search("idx_column_t2_s", 0, "Default", "test", "TestOpenColumn2", true, true, 1, $$42, 1, $$42, true, true, true) [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|
-                                      unnest-map [$$51, $$52] <- index-search("idx_column_t2_s", 0, "Default", "test", "TestOpenColumn2", true, true, 1, $$42, 1, $$42, true, true, true) [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]
-                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                          project ([$$47, $$42]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            assign [$$42] <- [to-string($$t1.getField("c_s"))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- ASSIGN  |PARTITIONED|
-                                              select (eq($$t1.getField("c_i64"), 2)) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                assign [$$47] <- [$$t1.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- ASSIGN  |PARTITIONED|
-                                                  project ([$$t1]) [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 []<-[$$40, $$t1] <- test.TestOpenRow1 [cardinality: 1.0, op-cost: 1.0, total-cost: 1.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|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      assign [$$42] <- [to-string($$t1.getField("c_s"))] project: [$$47, $$42] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- ASSIGN  |PARTITIONED|
+                                        select (eq($$t1.getField("c_i64"), 2)) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          assign [$$47] <- [$$t1.getField("c_x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- ASSIGN  |PARTITIONED|
+                                            project ([$$t1]) [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 []<-[$$40, $$t1] <- test.TestOpenRow1 [cardinality: 1.0, op-cost: 1.0, total-cost: 1.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_cbo/column/pushdown/other-pushdowns/other-pushdowns.004.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.004.plan
index d5b5f9a..90cfae5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.004.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.004.plan
@@ -1,40 +1,36 @@
-distribute result [$$61] [cardinality: 2.67, op-cost: 0.0, total-cost: 12.0]
+distribute result [$$61] [cardinality: 2.1, op-cost: 0.0, total-cost: 12.0]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
-  exchange [cardinality: 2.67, op-cost: 0.0, total-cost: 12.0]
+  exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 12.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$61]) [cardinality: 2.67, op-cost: 0.0, total-cost: 12.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$61] <- [{"$1": $$66, "$2": $$67}] [cardinality: 2.67, op-cost: 0.0, total-cost: 12.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$66, $$67] <- [agg-global-sql-sum($$70), agg-global-sql-sum($$71)] [cardinality: 2.67, op-cost: 0.0, total-cost: 12.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 2.67, op-cost: 0.0, total-cost: 12.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$70, $$71] <- [agg-local-sql-sum($$54), agg-local-sql-sum($$59)] [cardinality: 2.67, op-cost: 0.0, total-cost: 12.0]
-            -- AGGREGATE  |PARTITIONED|
-              project ([$$54, $$59]) [cardinality: 2.67, op-cost: 0.0, total-cost: 12.0]
+    assign [$$61] <- [{"$1": $$66, "$2": $$67}] project: [$$61] [cardinality: 2.1, op-cost: 0.0, total-cost: 12.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$66, $$67] <- [agg-global-sql-sum($$70), agg-global-sql-sum($$71)] [cardinality: 2.1, op-cost: 0.0, total-cost: 12.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 12.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$70, $$71] <- [agg-local-sql-sum($$54), agg-local-sql-sum($$59)] [cardinality: 2.1, op-cost: 0.0, total-cost: 12.0]
+          -- AGGREGATE  |PARTITIONED|
+            assign [$$59, $$54] <- [object-length($$p2), object-length($$p1)] project: [$$54, $$59] [cardinality: 2.1, op-cost: 0.0, total-cost: 12.0]
+            -- ASSIGN  |PARTITIONED|
+              project ([$$p1, $$p2]) [cardinality: 2.1, op-cost: 0.0, total-cost: 12.0]
               -- STREAM_PROJECT  |PARTITIONED|
-                assign [$$59, $$54] <- [object-length($$p2), object-length($$p1)] [cardinality: 2.67, op-cost: 0.0, total-cost: 12.0]
-                -- ASSIGN  |PARTITIONED|
-                  project ([$$p1, $$p2]) [cardinality: 2.67, op-cost: 0.0, total-cost: 12.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    exchange [cardinality: 2.67, op-cost: 0.0, total-cost: 12.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      join (eq($$63, $$62)) [cardinality: 2.67, op-cost: 4.0, total-cost: 12.0]
-                      -- HYBRID_HASH_JOIN [$$62][$$63]  |PARTITIONED|
-                        exchange [cardinality: 2.0, op-cost: 2.0, total-cost: 4.0]
-                        -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
-                          data-scan []<-[$$62, $$p1] <- test.ColumnDataset [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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: 2.0, op-cost: 2.0, total-cost: 4.0]
-                        -- HASH_PARTITION_EXCHANGE [$$63]  |PARTITIONED|
-                          data-scan []<-[$$63, $$p2] <- test.RowDataset [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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: 2.1, op-cost: 0.0, total-cost: 12.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  join (eq($$63, $$62)) [cardinality: 2.1, op-cost: 4.0, total-cost: 12.0]
+                  -- HYBRID_HASH_JOIN [$$62][$$63]  |PARTITIONED|
+                    exchange [cardinality: 2.0, op-cost: 2.0, total-cost: 4.0]
+                    -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                      data-scan []<-[$$62, $$p1] <- test.ColumnDataset [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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: 2.0, op-cost: 2.0, total-cost: 4.0]
+                    -- HASH_PARTITION_EXCHANGE [$$63]  |PARTITIONED|
+                      data-scan []<-[$$63, $$p2] <- test.RowDataset [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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_cbo/column/pushdown/other-pushdowns/other-pushdowns.008.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.008.plan
index 95a0f82..a32f4ad 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.008.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.008.plan
@@ -1,52 +1,44 @@
-distribute result [$$69] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+distribute result [$$69] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+  exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$69]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$69] <- [{"text": $$text, "$1": $$73}] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-        -- SORT_MERGE_EXCHANGE [$$text(ASC) ]  |PARTITIONED|
-          group by ([$$text := $$81]) decor ([]) {
-                    aggregate [$$73] <- [agg-global-sql-sum($$80)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-          -- SORT_GROUP_BY[$$81]  |PARTITIONED|
-            exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-            -- HASH_PARTITION_EXCHANGE [$$81]  |PARTITIONED|
-              group by ([$$81 := $$71]) decor ([]) {
-                        aggregate [$$80] <- [agg-local-sql-sum(sql-count($$75))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-              -- SORT_GROUP_BY[$$71]  |PARTITIONED|
-                exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  select (eq(lowercase($$71), "string")) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                  -- STREAM_SELECT  |PARTITIONED|
-                    project ([$$75, $$71]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$71] <- [$$ht.getField("display_url")] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+    assign [$$69] <- [{"text": $$text, "$1": $$73}] project: [$$69] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+      -- SORT_MERGE_EXCHANGE [$$text(ASC) ]  |PARTITIONED|
+        group by ([$$text := $$81]) decor ([]) {
+                  aggregate [$$73] <- [agg-global-sql-sum($$80)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+        -- SORT_GROUP_BY[$$81]  |PARTITIONED|
+          exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+          -- HASH_PARTITION_EXCHANGE [$$81]  |PARTITIONED|
+            group by ([$$81 := $$71]) decor ([]) {
+                      aggregate [$$80] <- [agg-local-sql-sum(sql-count($$75))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+            -- SORT_GROUP_BY[$$71]  |PARTITIONED|
+              exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                select (eq(lowercase($$71), "string")) [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+                -- STREAM_SELECT  |PARTITIONED|
+                  assign [$$71] <- [$$ht.getField("display_url")] project: [$$75, $$71] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                  -- ASSIGN  |PARTITIONED|
+                    unnest $$ht <- scan-collection($$75) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                    -- UNNEST  |PARTITIONED|
+                      assign [$$75] <- [$$p1.getField("entities").getField("urls")] project: [$$75] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                       -- ASSIGN  |PARTITIONED|
-                        unnest $$ht <- scan-collection($$75) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                        -- UNNEST  |PARTITIONED|
-                          project ([$$75]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            assign [$$75] <- [$$p1.getField("entities").getField("urls")] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                            -- ASSIGN  |PARTITIONED|
-                              project ([$$p1]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                select (neq($$72, "0")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    data-scan []<-[$$72, $$p1] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) filter on: eq(lowercase(scan-collection($$p1.getField("entities").getField("urls")).getField("display_url")), "string") [cardinality: 1.0, op-cost: 1.0, total-cost: 1.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|
+                        select (neq($$72, "0")) project: [$$p1] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- STREAM_SELECT  |PARTITIONED|
+                          exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            data-scan []<-[$$72, $$p1] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) filter on: eq(lowercase(scan-collection($$p1.getField("entities").getField("urls")).getField("display_url")), "string") [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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_cbo/column/pushdown/other-pushdowns/other-pushdowns.010.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.010.plan
index e5795d56..d8123a1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.010.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.010.plan
@@ -1,54 +1,44 @@
-distribute result [$$68] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+distribute result [$$68] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+  exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$68]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$68] <- [{"text": $$text, "$1": $$72}] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-        -- SORT_MERGE_EXCHANGE [$$text(ASC) ]  |PARTITIONED|
-          group by ([$$text := $$79]) decor ([]) {
-                    aggregate [$$72] <- [agg-global-sql-sum($$78)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-          -- SORT_GROUP_BY[$$79]  |PARTITIONED|
-            exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-            -- HASH_PARTITION_EXCHANGE [$$79]  |PARTITIONED|
-              group by ([$$79 := $$70]) decor ([]) {
-                        aggregate [$$78] <- [agg-local-sql-sum(sql-sum($$75))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-              -- SORT_GROUP_BY[$$70]  |PARTITIONED|
-                exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  select (eq(lowercase($$70), "string")) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                  -- STREAM_SELECT  |PARTITIONED|
-                    project ([$$75, $$70]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$75, $$70] <- [$$ht.getField("indices"), $$ht.getField("display_url")] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
+    assign [$$68] <- [{"text": $$text, "$1": $$72}] project: [$$68] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+      -- SORT_MERGE_EXCHANGE [$$text(ASC) ]  |PARTITIONED|
+        group by ([$$text := $$79]) decor ([]) {
+                  aggregate [$$72] <- [agg-global-sql-sum($$78)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+        -- SORT_GROUP_BY[$$79]  |PARTITIONED|
+          exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+          -- HASH_PARTITION_EXCHANGE [$$79]  |PARTITIONED|
+            group by ([$$79 := $$70]) decor ([]) {
+                      aggregate [$$78] <- [agg-local-sql-sum(sql-sum($$75))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+            -- SORT_GROUP_BY[$$70]  |PARTITIONED|
+              exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                select (eq(lowercase($$70), "string")) [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+                -- STREAM_SELECT  |PARTITIONED|
+                  assign [$$75, $$70] <- [$$ht.getField("indices"), $$ht.getField("display_url")] project: [$$75, $$70] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                  -- ASSIGN  |PARTITIONED|
+                    unnest $$ht <- scan-collection($$74) project: [$$ht] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                    -- UNNEST  |PARTITIONED|
+                      assign [$$74] <- [$$p1.getField("entities").getField("urls")] project: [$$74] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                       -- ASSIGN  |PARTITIONED|
-                        project ([$$ht]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          unnest $$ht <- scan-collection($$74) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                          -- UNNEST  |PARTITIONED|
-                            project ([$$74]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              assign [$$74] <- [$$p1.getField("entities").getField("urls")] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                              -- ASSIGN  |PARTITIONED|
-                                project ([$$p1]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  select (neq($$71, "0")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- STREAM_SELECT  |PARTITIONED|
-                                    exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      data-scan []<-[$$71, $$p1] <- test.ColumnDataset project ({entities:{urls:[{display_url:any,indices:any}]}}) filter on: eq(lowercase(scan-collection($$p1.getField("entities").getField("urls")).getField("display_url")), "string") [cardinality: 1.0, op-cost: 1.0, total-cost: 1.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|
+                        select (neq($$71, "0")) project: [$$p1] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- STREAM_SELECT  |PARTITIONED|
+                          exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            data-scan []<-[$$71, $$p1] <- test.ColumnDataset project ({entities:{urls:[{display_url:any,indices:any}]}}) filter on: eq(lowercase(scan-collection($$p1.getField("entities").getField("urls")).getField("display_url")), "string") [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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_cbo/column/pushdown/other-pushdowns/other-pushdowns.012.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.012.plan
index 56818bf..37d5ca6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.012.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.012.plan
@@ -2,33 +2,27 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$22]) [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$22] <- [{"display_url": $$27}] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$27]) [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
-          -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
-            order (ASC, $$25) [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
-            -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
-              exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$27, $$25]) [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$27] <- [get-item($$24, 0).getField("display_url")] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+    assign [$$22] <- [{"display_url": $$27}] project: [$$22] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$27]) [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+        -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
+          order (ASC, $$25) [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+          -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+            exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$27] <- [get-item($$24, 0).getField("display_url")] project: [$$27, $$25] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+              -- ASSIGN  |PARTITIONED|
+                select (eq(sql-count($$24), 1)) [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+                -- STREAM_SELECT  |PARTITIONED|
+                  assign [$$24] <- [$$p.getField("entities").getField("urls")] project: [$$25, $$24] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                   -- ASSIGN  |PARTITIONED|
-                    select (eq(sql-count($$24), 1)) [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
-                    -- STREAM_SELECT  |PARTITIONED|
-                      project ([$$25, $$24]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$24] <- [$$p.getField("entities").getField("urls")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                        -- ASSIGN  |PARTITIONED|
-                          exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            data-scan []<-[$$25, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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: 2.0, op-cost: 0.0, total-cost: 2.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$25, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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_cbo/column/pushdown/other-pushdowns/other-pushdowns.014.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.014.plan
index 4ec8540..f4a7046 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.014.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.014.plan
@@ -1,111 +1,95 @@
-distribute result [$$101] [cardinality: 0.0, op-cost: 0.0, total-cost: 5.5]
+distribute result [$$101] [cardinality: 0.0, op-cost: 0.0, total-cost: 6.5]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 5.5]
+  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 6.5]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$101]) [cardinality: 0.0, op-cost: 0.0, total-cost: 5.5]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$101] <- [{"uname": $$uname, "cnt": $$105}] [cardinality: 0.0, op-cost: 0.0, total-cost: 5.5]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 5.5]
-        -- SORT_MERGE_EXCHANGE [$$105(DESC), $$uname(ASC) ]  |PARTITIONED|
-          order (DESC, $$105) (ASC, $$uname) [cardinality: 0.0, op-cost: 0.0, total-cost: 5.5]
-          -- STABLE_SORT [$$105(DESC), $$uname(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 5.5]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              group by ([$$uname := $$116]) decor ([]) {
-                        aggregate [$$105] <- [agg-sql-sum($$115)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 5.5]
-              -- SORT_GROUP_BY[$$116]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 5.5]
-                -- HASH_PARTITION_EXCHANGE [$$116]  |PARTITIONED|
-                  group by ([$$116 := $$102]) decor ([]) {
-                            aggregate [$$115] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- AGGREGATE  |LOCAL|
-                              nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                         } [cardinality: 0.0, op-cost: 0.0, total-cost: 5.5]
-                  -- SORT_GROUP_BY[$$102]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 5.5]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      project ([$$102]) [cardinality: 0.0, op-cost: 0.0, total-cost: 5.5]
+    assign [$$101] <- [{"uname": $$uname, "cnt": $$105}] project: [$$101] [cardinality: 0.0, op-cost: 0.0, total-cost: 6.5]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 6.5]
+      -- SORT_MERGE_EXCHANGE [$$105(DESC), $$uname(ASC) ]  |PARTITIONED|
+        order (DESC, $$105) (ASC, $$uname) [cardinality: 0.0, op-cost: 0.0, total-cost: 6.5]
+        -- STABLE_SORT [$$105(DESC), $$uname(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 6.5]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            group by ([$$uname := $$116]) decor ([]) {
+                      aggregate [$$105] <- [agg-sql-sum($$115)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 6.5]
+            -- SORT_GROUP_BY[$$116]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 6.5]
+              -- HASH_PARTITION_EXCHANGE [$$116]  |PARTITIONED|
+                group by ([$$116 := $$102]) decor ([]) {
+                          aggregate [$$115] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- AGGREGATE  |LOCAL|
+                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 6.5]
+                -- SORT_GROUP_BY[$$102]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 6.5]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    select ($$92) project: [$$102] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- STREAM_SELECT  |PARTITIONED|
+                      project ([$$92, $$102]) [cardinality: 0.0, op-cost: 0.0, total-cost: 6.5]
                       -- STREAM_PROJECT  |PARTITIONED|
-                        select ($$92) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_SELECT  |PARTITIONED|
-                          project ([$$92, $$102]) [cardinality: 0.0, op-cost: 0.0, total-cost: 5.5]
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 5.5]
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 6.5]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          group by ([$$114 := $$112]) decor ([$$102]) {
+                                    aggregate [$$92] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- AGGREGATE  |LOCAL|
+                                      select (not(is-missing($$113))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- STREAM_SELECT  |LOCAL|
+                                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                 } [cardinality: 0.0, op-cost: 2.25, total-cost: 6.5]
+                          -- PRE_CLUSTERED_GROUP_BY[$$112]  |PARTITIONED|
+                            exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 4.25]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              group by ([$$114 := $$112]) decor ([$$102]) {
-                                        aggregate [$$92] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- AGGREGATE  |LOCAL|
-                                          select (not(is-missing($$113))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- STREAM_SELECT  |LOCAL|
-                                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                     } [cardinality: 0.0, op-cost: 2.25, total-cost: 5.5]
-                              -- PRE_CLUSTERED_GROUP_BY[$$112]  |PARTITIONED|
-                                exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 3.25]
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  order (ASC, $$112) [cardinality: 1.0, op-cost: 2.25, total-cost: 3.25]
-                                  -- STABLE_SORT [$$112(ASC)]  |PARTITIONED|
-                                    exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                                    -- HASH_PARTITION_EXCHANGE [$$112]  |PARTITIONED|
-                                      project ([$$102, $$113, $$112]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          left outer join (eq($$106, $$88)) [cardinality: 2.1, op-cost: 2.0, total-cost: 6.0]
-                                          -- HYBRID_HASH_JOIN [$$106][$$88]  |PARTITIONED|
-                                            exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                                            -- HASH_PARTITION_EXCHANGE [$$106]  |PARTITIONED|
-                                              running-aggregate [$$112] <- [create-query-uid()] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                                              -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                project ([$$102, $$106]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  assign [$$106] <- [$$ht1.getField("display_url")] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    project ([$$102, $$ht1]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      unnest $$ht1 <- scan-collection($$107) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                                                      -- UNNEST  |PARTITIONED|
-                                                        project ([$$107, $$102]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          assign [$$107, $$102] <- [$$p1.getField("entities").getField("urls"), $$p1.getField("user").getField("name")] [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            project ([$$p1]) [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 1.0]
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                data-scan []<-[$$103, $$p1] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]},user:{name:any}}) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.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 [$$88]  |PARTITIONED|
-                                              project ([$$113, $$88]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                assign [$$113, $$88] <- [true, $$ht2.getField("display_url")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              order (ASC, $$112) [cardinality: 2.0, op-cost: 2.25, total-cost: 4.25]
+                              -- STABLE_SORT [$$112(ASC)]  |PARTITIONED|
+                                exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                -- HASH_PARTITION_EXCHANGE [$$112]  |PARTITIONED|
+                                  project ([$$102, $$113, $$112]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      left outer join (eq($$106, $$88)) [cardinality: 2.1, op-cost: 2.0, total-cost: 8.0]
+                                      -- HYBRID_HASH_JOIN [$$106][$$88]  |PARTITIONED|
+                                        exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                        -- HASH_PARTITION_EXCHANGE [$$106]  |PARTITIONED|
+                                          running-aggregate [$$112] <- [create-query-uid()] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                          -- RUNNING_AGGREGATE  |PARTITIONED|
+                                            assign [$$106] <- [$$ht1.getField("display_url")] project: [$$102, $$106] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                            -- ASSIGN  |PARTITIONED|
+                                              unnest $$ht1 <- scan-collection($$107) project: [$$102, $$ht1] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                              -- UNNEST  |PARTITIONED|
+                                                assign [$$107, $$102] <- [$$p1.getField("entities").getField("urls"), $$p1.getField("user").getField("name")] project: [$$107, $$102] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                                                 -- ASSIGN  |PARTITIONED|
-                                                  project ([$$ht2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                  project ([$$p1]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                                                   -- STREAM_PROJECT  |PARTITIONED|
-                                                    unnest $$ht2 <- scan-collection($$108) [cardinality: 1.0, op-cost: 1.0, total-cost: 1.0]
-                                                    -- UNNEST  |PARTITIONED|
-                                                      project ([$$108]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        assign [$$108] <- [$$p2.getField("entities").getField("urls")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          project ([$$p2]) [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 []<-[$$104, $$p2] <- test.RowDataset [cardinality: 1.0, op-cost: 1.0, total-cost: 1.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: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      data-scan []<-[$$103, $$p1] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]},user:{name:any}}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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 [$$88]  |PARTITIONED|
+                                          assign [$$113, $$88] <- [true, $$ht2.getField("display_url")] project: [$$113, $$88] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- ASSIGN  |PARTITIONED|
+                                            unnest $$ht2 <- scan-collection($$108) project: [$$ht2] [cardinality: 1.0, op-cost: 2.0, total-cost: 2.0]
+                                            -- UNNEST  |PARTITIONED|
+                                              assign [$$108] <- [$$p2.getField("entities").getField("urls")] project: [$$108] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- ASSIGN  |PARTITIONED|
+                                                project ([$$p2]) [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 []<-[$$104, $$p2] <- test.RowDataset [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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_cbo/column/pushdown/other-pushdowns/other-pushdowns.015.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.015.plan
index dbbed89..1111ccc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.015.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.015.plan
@@ -1,54 +1,48 @@
-distribute result [$$52] [cardinality: 2.67, op-cost: 0.0, total-cost: 12.0]
+distribute result [$$52] [cardinality: 4.0, op-cost: 0.0, total-cost: 12.0]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
-  exchange [cardinality: 2.67, op-cost: 0.0, total-cost: 12.0]
+  exchange [cardinality: 4.0, op-cost: 0.0, total-cost: 12.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$52]) [cardinality: 2.67, op-cost: 0.0, total-cost: 12.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$52] <- [{"$1": $$57}] [cardinality: 2.67, op-cost: 0.0, total-cost: 12.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$57] <- [agg-sql-sum($$60)] [cardinality: 2.67, op-cost: 0.0, total-cost: 12.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 2.67, op-cost: 0.0, total-cost: 12.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$60] <- [agg-sql-count($$50)] [cardinality: 2.67, op-cost: 0.0, total-cost: 12.0]
-            -- AGGREGATE  |PARTITIONED|
-              project ([$$50]) [cardinality: 2.67, op-cost: 0.0, total-cost: 12.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                exchange [cardinality: 2.67, op-cost: 0.0, total-cost: 12.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  join (eq($$55, $$56)) [cardinality: 2.67, op-cost: 4.0, total-cost: 12.0]
-                  -- HYBRID_HASH_JOIN [$$55][$$56]  |PARTITIONED|
-                    exchange [cardinality: 2.0, op-cost: 2.0, total-cost: 4.0]
-                    -- HASH_PARTITION_EXCHANGE [$$55]  |PARTITIONED|
-                      assign [$$50] <- [$$55.getField("y")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+    assign [$$52] <- [{"$1": $$57}] project: [$$52] [cardinality: 4.0, op-cost: 0.0, total-cost: 12.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$57] <- [agg-sql-sum($$60)] [cardinality: 4.0, op-cost: 0.0, total-cost: 12.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 4.0, op-cost: 0.0, total-cost: 12.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$60] <- [agg-sql-count($$50)] [cardinality: 4.0, op-cost: 0.0, total-cost: 12.0]
+          -- AGGREGATE  |PARTITIONED|
+            project ([$$50]) [cardinality: 4.0, op-cost: 0.0, total-cost: 12.0]
+            -- STREAM_PROJECT  |PARTITIONED|
+              exchange [cardinality: 4.0, op-cost: 0.0, total-cost: 12.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                join (eq($$55, $$56)) [cardinality: 4.0, op-cost: 4.0, total-cost: 12.0]
+                -- HYBRID_HASH_JOIN [$$55][$$56]  |PARTITIONED|
+                  exchange [cardinality: 2.0, op-cost: 2.0, total-cost: 4.0]
+                  -- HASH_PARTITION_EXCHANGE [$$55]  |PARTITIONED|
+                    assign [$$50] <- [$$55.getField("y")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                    -- ASSIGN  |PARTITIONED|
+                      assign [$$55] <- [$$c.getField("x")] project: [$$55] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                       -- ASSIGN  |PARTITIONED|
-                        project ([$$55]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                        project ([$$c]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                         -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$55] <- [$$c.getField("x")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                          -- ASSIGN  |PARTITIONED|
-                            project ([$$c]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              exchange [cardinality: 2.0, op-cost: 2.0, total-cost: 4.0]
+                          exchange [cardinality: 2.0, op-cost: 2.0, total-cost: 4.0]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            data-scan []<-[$$53, $$c] <- test.ColumnDataset project ({x:any}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.0]
+                            -- DATASOURCE_SCAN  |PARTITIONED|
+                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                data-scan []<-[$$53, $$c] <- test.ColumnDataset project ({x:any}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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: 2.0, op-cost: 2.0, total-cost: 4.0]
-                    -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
-                      project ([$$56]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  exchange [cardinality: 2.0, op-cost: 2.0, total-cost: 4.0]
+                  -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
+                    assign [$$56] <- [$$r.getField("x")] project: [$$56] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                    -- ASSIGN  |PARTITIONED|
+                      project ([$$r]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$56] <- [$$r.getField("x")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                        -- ASSIGN  |PARTITIONED|
-                          project ([$$r]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            exchange [cardinality: 2.0, op-cost: 2.0, total-cost: 4.0]
+                        exchange [cardinality: 2.0, op-cost: 2.0, total-cost: 4.0]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$54, $$r] <- test.RowDataset [cardinality: 2.0, op-cost: 2.0, total-cost: 2.0]
+                          -- DATASOURCE_SCAN  |PARTITIONED|
+                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              data-scan []<-[$$54, $$r] <- test.RowDataset [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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|
+                              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_cbo/column/pushdown/other-pushdowns/other-pushdowns.016.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.016.plan
index a06989f..cd2903d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.016.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.016.plan
@@ -2,26 +2,37 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$65]) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$65] <- [{"$1": $$67}] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$67] <- [agg-sql-sum($$72)] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$72] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-            -- AGGREGATE  |PARTITIONED|
-              select (or($$53, $$57)) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$53, $$57]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                -- STREAM_PROJECT  |PARTITIONED|
+    assign [$$65] <- [{"$1": $$67}] project: [$$65] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$67] <- [agg-sql-sum($$72)] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$72] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+          -- AGGREGATE  |PARTITIONED|
+            select (or($$53, $$57)) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$53, $$57]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+              -- STREAM_PROJECT  |PARTITIONED|
+                subplan {
+                          aggregate [$$57] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- AGGREGATE  |LOCAL|
+                            select (eq($$71, 2)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- STREAM_SELECT  |LOCAL|
+                              assign [$$71] <- [$$e.getField("x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- ASSIGN  |LOCAL|
+                                unnest $$e <- scan-collection($$68) [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: 2.0, op-cost: 0.0, total-cost: 2.0]
+                -- SUBPLAN  |PARTITIONED|
                   subplan {
-                            aggregate [$$57] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            aggregate [$$53] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- AGGREGATE  |LOCAL|
-                              select (eq($$71, 2)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              select (eq($$69, 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                               -- STREAM_SELECT  |LOCAL|
-                                assign [$$71] <- [$$e.getField("x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                assign [$$69] <- [$$e.getField("x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                 -- ASSIGN  |LOCAL|
                                   unnest $$e <- scan-collection($$68) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                   -- UNNEST  |LOCAL|
@@ -29,30 +40,15 @@
                                     -- NESTED_TUPLE_SOURCE  |LOCAL|
                          } [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                   -- SUBPLAN  |PARTITIONED|
-                    subplan {
-                              aggregate [$$53] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- AGGREGATE  |LOCAL|
-                                select (eq($$69, 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |LOCAL|
-                                  assign [$$69] <- [$$e.getField("x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ASSIGN  |LOCAL|
-                                    unnest $$e <- scan-collection($$68) [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: 2.0, op-cost: 0.0, total-cost: 2.0]
-                    -- SUBPLAN  |PARTITIONED|
-                      project ([$$68]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                    assign [$$68] <- [$$c.getField("val1")] project: [$$68] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                    -- ASSIGN  |PARTITIONED|
+                      project ([$$c]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$68] <- [$$c.getField("val1")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                        -- ASSIGN  |PARTITIONED|
-                          project ([$$c]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                        exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$66, $$c] <- test.ColumnDataset project ({val1:[{x:any}]}) filter on: or(eq(scan-collection($$c.getField("val1")).getField("x"), 1), eq(scan-collection($$c.getField("val1")).getField("x"), 2)) range-filter on: or(eq(scan-collection($$c.getField("val1")).getField("x"), 1), eq(scan-collection($$c.getField("val1")).getField("x"), 2)) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.0]
+                          -- DATASOURCE_SCAN  |PARTITIONED|
+                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              data-scan []<-[$$66, $$c] <- test.ColumnDataset project ({val1:[{x:any}]}) filter on: or(eq(scan-collection($$c.getField("val1")).getField("x"), 1), eq(scan-collection($$c.getField("val1")).getField("x"), 2)) range-filter on: or(eq(scan-collection($$c.getField("val1")).getField("x"), 1), eq(scan-collection($$c.getField("val1")).getField("x"), 2)) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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|
+                              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_cbo/column/pushdown/other-pushdowns/other-pushdowns.017.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.017.plan
index d108091..459e217 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.017.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.017.plan
@@ -2,26 +2,37 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$65]) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$65] <- [{"$1": $$67}] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$67] <- [agg-sql-sum($$72)] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$72] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-            -- AGGREGATE  |PARTITIONED|
-              select (or($$53, $$57)) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$53, $$57]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                -- STREAM_PROJECT  |PARTITIONED|
+    assign [$$65] <- [{"$1": $$67}] project: [$$65] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$67] <- [agg-sql-sum($$72)] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$72] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+          -- AGGREGATE  |PARTITIONED|
+            select (or($$53, $$57)) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$53, $$57]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+              -- STREAM_PROJECT  |PARTITIONED|
+                subplan {
+                          aggregate [$$57] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- AGGREGATE  |LOCAL|
+                            select (eq($$71, 2)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- STREAM_SELECT  |LOCAL|
+                              assign [$$71] <- [$$e.getField("y")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- ASSIGN  |LOCAL|
+                                unnest $$e <- scan-collection($$68) [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: 2.0, op-cost: 0.0, total-cost: 2.0]
+                -- SUBPLAN  |PARTITIONED|
                   subplan {
-                            aggregate [$$57] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            aggregate [$$53] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- AGGREGATE  |LOCAL|
-                              select (eq($$71, 2)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              select (eq($$69, 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                               -- STREAM_SELECT  |LOCAL|
-                                assign [$$71] <- [$$e.getField("y")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                assign [$$69] <- [$$e.getField("x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                 -- ASSIGN  |LOCAL|
                                   unnest $$e <- scan-collection($$68) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                   -- UNNEST  |LOCAL|
@@ -29,30 +40,15 @@
                                     -- NESTED_TUPLE_SOURCE  |LOCAL|
                          } [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                   -- SUBPLAN  |PARTITIONED|
-                    subplan {
-                              aggregate [$$53] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- AGGREGATE  |LOCAL|
-                                select (eq($$69, 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |LOCAL|
-                                  assign [$$69] <- [$$e.getField("x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ASSIGN  |LOCAL|
-                                    unnest $$e <- scan-collection($$68) [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: 2.0, op-cost: 0.0, total-cost: 2.0]
-                    -- SUBPLAN  |PARTITIONED|
-                      project ([$$68]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                    assign [$$68] <- [$$c.getField("val1")] project: [$$68] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                    -- ASSIGN  |PARTITIONED|
+                      project ([$$c]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$68] <- [$$c.getField("val1")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                        -- ASSIGN  |PARTITIONED|
-                          project ([$$c]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                        exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$66, $$c] <- test.ColumnDataset project ({val1:[{x:any,y:any}]}) filter on: or(eq(scan-collection($$c.getField("val1")).getField("x"), 1), eq(scan-collection($$c.getField("val1")).getField("y"), 2)) range-filter on: or(eq(scan-collection($$c.getField("val1")).getField("x"), 1), eq(scan-collection($$c.getField("val1")).getField("y"), 2)) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.0]
+                          -- DATASOURCE_SCAN  |PARTITIONED|
+                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              data-scan []<-[$$66, $$c] <- test.ColumnDataset project ({val1:[{x:any,y:any}]}) filter on: or(eq(scan-collection($$c.getField("val1")).getField("x"), 1), eq(scan-collection($$c.getField("val1")).getField("y"), 2)) range-filter on: or(eq(scan-collection($$c.getField("val1")).getField("x"), 1), eq(scan-collection($$c.getField("val1")).getField("y"), 2)) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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|
+                              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_cbo/column/pushdown/other-pushdowns/other-pushdowns.018.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.018.plan
index 6f485c7..d38ffb1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.018.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.018.plan
@@ -2,59 +2,55 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$65]) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$65] <- [{"$1": $$67}] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$67] <- [agg-sql-sum($$72)] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$72] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-            -- AGGREGATE  |PARTITIONED|
-              select (or($$53, $$57)) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$53, $$57]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  subplan {
-                            aggregate [$$57] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- AGGREGATE  |LOCAL|
-                              select (eq($$71, 2)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_SELECT  |LOCAL|
-                                assign [$$71] <- [$$e.getField("y")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- ASSIGN  |LOCAL|
-                                  unnest $$e <- scan-collection($$70) [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: 2.0, op-cost: 0.0, total-cost: 2.0]
-                  -- SUBPLAN  |PARTITIONED|
-                    project ([$$70, $$53]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      subplan {
-                                aggregate [$$53] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- AGGREGATE  |LOCAL|
-                                  select (eq($$69, 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- STREAM_SELECT  |LOCAL|
-                                    assign [$$69] <- [$$e.getField("x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- ASSIGN  |LOCAL|
-                                      unnest $$e <- scan-collection($$68) [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: 2.0, op-cost: 0.0, total-cost: 2.0]
-                      -- SUBPLAN  |PARTITIONED|
-                        project ([$$70, $$68]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+    assign [$$65] <- [{"$1": $$67}] project: [$$65] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$67] <- [agg-sql-sum($$72)] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$72] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+          -- AGGREGATE  |PARTITIONED|
+            select (or($$53, $$57)) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              project ([$$53, $$57]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+              -- STREAM_PROJECT  |PARTITIONED|
+                subplan {
+                          aggregate [$$57] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- AGGREGATE  |LOCAL|
+                            select (eq($$71, 2)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- STREAM_SELECT  |LOCAL|
+                              assign [$$71] <- [$$e.getField("y")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- ASSIGN  |LOCAL|
+                                unnest $$e <- scan-collection($$70) [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: 2.0, op-cost: 0.0, total-cost: 2.0]
+                -- SUBPLAN  |PARTITIONED|
+                  project ([$$70, $$53]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    subplan {
+                              aggregate [$$53] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- AGGREGATE  |LOCAL|
+                                select (eq($$69, 1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- STREAM_SELECT  |LOCAL|
+                                  assign [$$69] <- [$$e.getField("x")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- ASSIGN  |LOCAL|
+                                    unnest $$e <- scan-collection($$68) [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: 2.0, op-cost: 0.0, total-cost: 2.0]
+                    -- SUBPLAN  |PARTITIONED|
+                      assign [$$70, $$68] <- [$$c.getField("val2"), $$c.getField("val1")] project: [$$70, $$68] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                      -- ASSIGN  |PARTITIONED|
+                        project ([$$c]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                         -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$70, $$68] <- [$$c.getField("val2"), $$c.getField("val1")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                          -- ASSIGN  |PARTITIONED|
-                            project ([$$c]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                          exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            data-scan []<-[$$66, $$c] <- test.ColumnDataset project ({val2:[{y:any}],val1:[{x:any}]}) filter on: or(eq(scan-collection($$c.getField("val1")).getField("x"), 1), eq(scan-collection($$c.getField("val2")).getField("y"), 2)) range-filter on: or(eq(scan-collection($$c.getField("val1")).getField("x"), 1), eq(scan-collection($$c.getField("val2")).getField("y"), 2)) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.0]
+                            -- DATASOURCE_SCAN  |PARTITIONED|
+                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                data-scan []<-[$$66, $$c] <- test.ColumnDataset project ({val2:[{y:any}],val1:[{x:any}]}) filter on: or(eq(scan-collection($$c.getField("val1")).getField("x"), 1), eq(scan-collection($$c.getField("val2")).getField("y"), 2)) range-filter on: or(eq(scan-collection($$c.getField("val1")).getField("x"), 1), eq(scan-collection($$c.getField("val2")).getField("y"), 2)) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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|
+                                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_cbo/column/pushdown/other-pushdowns/other-pushdowns.019.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.019.plan
index fb96640..02a9cd5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.019.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.019.plan
@@ -2,23 +2,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$31]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$31] <- [{"$1": get-item(get-item(get-item($$32, 0).getField("f2"), 0), 0), "$2": get-item($$47.getField("f3"), 1), "$3": get-item(get-item($$47.getField("f2"), 1), 1)}] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+    assign [$$31] <- [{"$1": get-item(get-item(get-item($$32, 0).getField("f2"), 0), 0), "$2": get-item($$47.getField("f3"), 1), "$3": get-item(get-item($$47.getField("f2"), 1), 1)}] project: [$$31] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+    -- ASSIGN  |PARTITIONED|
+      assign [$$47] <- [get-item($$32, 1)] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
       -- ASSIGN  |PARTITIONED|
-        assign [$$47] <- [get-item($$32, 1)] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+        assign [$$32] <- [$$c.getField("f1")] project: [$$32] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
         -- ASSIGN  |PARTITIONED|
-          project ([$$32]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+          project ([$$c]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
           -- STREAM_PROJECT  |PARTITIONED|
-            assign [$$32] <- [$$c.getField("f1")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-            -- ASSIGN  |PARTITIONED|
-              project ([$$c]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+            exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$33, $$c] <- test.ColumnDataset project ({f1:[{f2:[[any]],f3:[any]}]}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.0]
+              -- DATASOURCE_SCAN  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  data-scan []<-[$$33, $$c] <- test.ColumnDataset project ({f1:[{f2:[[any]],f3:[any]}]}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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|
+                  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_cbo/column/pushdown/other-pushdowns/other-pushdowns.020.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.020.plan
index 2ada1fb..0dfc527 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.020.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.020.plan
@@ -2,50 +2,44 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$66]) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$66] <- [{"count": $$69}] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$69] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-          -- SORT_MERGE_EXCHANGE [$$64(ASC) ]  |PARTITIONED|
-            distinct ([$$64]) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (ASC, $$64) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-                -- STABLE_SORT [$$64(ASC)]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-                  -- HASH_PARTITION_EXCHANGE [$$64]  |PARTITIONED|
-                    project ([$$64]) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+    assign [$$66] <- [{"count": $$69}] project: [$$66] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$69] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+        -- SORT_MERGE_EXCHANGE [$$64(ASC) ]  |PARTITIONED|
+          distinct ([$$64]) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              order (ASC, $$64) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+              -- STABLE_SORT [$$64(ASC)]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+                -- HASH_PARTITION_EXCHANGE [$$64]  |PARTITIONED|
+                  select (ge($$68, 3)) project: [$$64] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+                  -- STREAM_SELECT  |PARTITIONED|
+                    project ([$$64, $$68]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                     -- STREAM_PROJECT  |PARTITIONED|
-                      select (ge($$68, 3)) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-                      -- STREAM_SELECT  |PARTITIONED|
-                        project ([$$64, $$68]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          subplan {
-                                    aggregate [$$68] <- [agg-sql-sum($$52)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- AGGREGATE  |LOCAL|
-                                      assign [$$52] <- [$$i.getField("count")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- ASSIGN  |LOCAL|
-                                        unnest $$i <- scan-collection($$70) [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: 2.0, op-cost: 0.0, total-cost: 2.0]
-                          -- SUBPLAN  |PARTITIONED|
-                            project ([$$64, $$70]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              assign [$$64, $$70] <- [to-object-var-str($$t).getField(0), $$t.getField("a1")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                              -- ASSIGN  |PARTITIONED|
-                                project ([$$t]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    data-scan []<-[$$67, $$t] <- test.ColumnDataset [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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|
+                      subplan {
+                                aggregate [$$68] <- [agg-sql-sum($$52)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- AGGREGATE  |LOCAL|
+                                  assign [$$52] <- [$$i.getField("count")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- ASSIGN  |LOCAL|
+                                    unnest $$i <- scan-collection($$70) [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: 2.0, op-cost: 0.0, total-cost: 2.0]
+                      -- SUBPLAN  |PARTITIONED|
+                        assign [$$64, $$70] <- [to-object-var-str($$t).getField(0), $$t.getField("a1")] project: [$$64, $$70] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                        -- ASSIGN  |PARTITIONED|
+                          project ([$$t]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              data-scan []<-[$$67, $$t] <- test.ColumnDataset [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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_cbo/column/pushdown/other-pushdowns/other-pushdowns.021.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.021.plan
index 2ada1fb..0dfc527 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.021.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.021.plan
@@ -2,50 +2,44 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$66]) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$66] <- [{"count": $$69}] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$69] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-          -- SORT_MERGE_EXCHANGE [$$64(ASC) ]  |PARTITIONED|
-            distinct ([$$64]) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-            -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (ASC, $$64) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-                -- STABLE_SORT [$$64(ASC)]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-                  -- HASH_PARTITION_EXCHANGE [$$64]  |PARTITIONED|
-                    project ([$$64]) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+    assign [$$66] <- [{"count": $$69}] project: [$$66] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$69] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+        -- SORT_MERGE_EXCHANGE [$$64(ASC) ]  |PARTITIONED|
+          distinct ([$$64]) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              order (ASC, $$64) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+              -- STABLE_SORT [$$64(ASC)]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+                -- HASH_PARTITION_EXCHANGE [$$64]  |PARTITIONED|
+                  select (ge($$68, 3)) project: [$$64] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+                  -- STREAM_SELECT  |PARTITIONED|
+                    project ([$$64, $$68]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                     -- STREAM_PROJECT  |PARTITIONED|
-                      select (ge($$68, 3)) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-                      -- STREAM_SELECT  |PARTITIONED|
-                        project ([$$64, $$68]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          subplan {
-                                    aggregate [$$68] <- [agg-sql-sum($$52)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- AGGREGATE  |LOCAL|
-                                      assign [$$52] <- [$$i.getField("count")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- ASSIGN  |LOCAL|
-                                        unnest $$i <- scan-collection($$70) [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: 2.0, op-cost: 0.0, total-cost: 2.0]
-                          -- SUBPLAN  |PARTITIONED|
-                            project ([$$64, $$70]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              assign [$$64, $$70] <- [to-object-var-str($$t).getField(0), $$t.getField("a1")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                              -- ASSIGN  |PARTITIONED|
-                                project ([$$t]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    data-scan []<-[$$67, $$t] <- test.ColumnDataset [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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|
+                      subplan {
+                                aggregate [$$68] <- [agg-sql-sum($$52)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- AGGREGATE  |LOCAL|
+                                  assign [$$52] <- [$$i.getField("count")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- ASSIGN  |LOCAL|
+                                    unnest $$i <- scan-collection($$70) [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: 2.0, op-cost: 0.0, total-cost: 2.0]
+                      -- SUBPLAN  |PARTITIONED|
+                        assign [$$64, $$70] <- [to-object-var-str($$t).getField(0), $$t.getField("a1")] project: [$$64, $$70] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                        -- ASSIGN  |PARTITIONED|
+                          project ([$$t]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              data-scan []<-[$$67, $$t] <- test.ColumnDataset [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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_cbo/column/pushdown/other-pushdowns/other-pushdowns.022.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.022.plan
index 9997dae..1975af3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.022.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.022.plan
@@ -4,65 +4,59 @@
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit 5 [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$84]) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-      -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$84] <- [{"name": $$93, "phone": $$94, "num_reviews": sql-count($$91), "overall_avg": $#2, "overall_reviews": $$83}] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-        -- ASSIGN  |PARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-          -- SORT_MERGE_EXCHANGE [$#2(DESC) ]  |PARTITIONED|
-            limit 5 [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-            -- STREAM_LIMIT  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (topK: 5) (DESC, $#2) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-                -- STABLE_SORT [topK: 5] [$#2(DESC)]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    subplan {
-                              aggregate [$$83] <- [listify($$82)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- AGGREGATE  |LOCAL|
-                                assign [$$82] <- [$$98.getField("Overall")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      assign [$$84] <- [{"name": $$93, "phone": $$94, "num_reviews": sql-count($$91), "overall_avg": $#2, "overall_reviews": $$83}] project: [$$84] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+      -- ASSIGN  |PARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+        -- SORT_MERGE_EXCHANGE [$#2(DESC) ]  |PARTITIONED|
+          limit 5 [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+          -- STREAM_LIMIT  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              order (topK: 5) (DESC, $#2) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+              -- STABLE_SORT [topK: 5] [$#2(DESC)]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  subplan {
+                            aggregate [$$83] <- [listify($$82)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- AGGREGATE  |LOCAL|
+                              assign [$$82] <- [$$98.getField("Overall")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- ASSIGN  |LOCAL|
+                                assign [$$98] <- [$$reviews.getField("ratings")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                 -- ASSIGN  |LOCAL|
-                                  assign [$$98] <- [$$reviews.getField("ratings")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ASSIGN  |LOCAL|
-                                    unnest $$reviews <- scan-collection($$91) [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: 0.0, op-cost: 0.0, total-cost: 2.0]
-                    -- SUBPLAN  |PARTITIONED|
-                      project ([$$93, $$94, $$91, $#2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$#2] <- [get-item($$72, 0)] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-                        -- ASSIGN  |PARTITIONED|
-                          subplan {
-                                    aggregate [$$72] <- [listify($$86)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- AGGREGATE  |LOCAL|
-                                      aggregate [$$86] <- [agg-sql-min($$69)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- AGGREGATE  |LOCAL|
-                                        assign [$$69] <- [$$97.getField("Overall")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- ASSIGN  |LOCAL|
-                                          assign [$$97] <- [$$89.getField("ratings")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- ASSIGN  |LOCAL|
-                                            unnest $$89 <- scan-collection($$91) [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: 0.0, op-cost: 0.0, total-cost: 2.0]
-                          -- SUBPLAN  |PARTITIONED|
-                            project ([$$93, $$94, $$91]) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+                                  unnest $$reviews <- scan-collection($$91) [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: 0.0, op-cost: 0.0, total-cost: 2.0]
+                  -- SUBPLAN  |PARTITIONED|
+                    assign [$#2] <- [get-item($$72, 0)] project: [$$93, $$94, $$91, $#2] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+                    -- ASSIGN  |PARTITIONED|
+                      subplan {
+                                aggregate [$$72] <- [listify($$86)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- AGGREGATE  |LOCAL|
+                                  aggregate [$$86] <- [agg-sql-min($$69)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- AGGREGATE  |LOCAL|
+                                    assign [$$69] <- [$$97.getField("Overall")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- ASSIGN  |LOCAL|
+                                      assign [$$97] <- [$$89.getField("ratings")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- ASSIGN  |LOCAL|
+                                        unnest $$89 <- scan-collection($$91) [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: 0.0, op-cost: 0.0, total-cost: 2.0]
+                      -- SUBPLAN  |PARTITIONED|
+                        select (eq($$ht.getField("city"), "Los Angeles")) project: [$$93, $$94, $$91] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+                        -- STREAM_SELECT  |PARTITIONED|
+                          assign [$$94, $$91, $$93] <- [$$ht.getField("phone"), $$ht.getField("reviews"), $$ht.getField("name")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                          -- ASSIGN  |PARTITIONED|
+                            project ([$$ht]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                             -- STREAM_PROJECT  |PARTITIONED|
-                              select (eq($$ht.getField("city"), "Los Angeles")) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-                              -- STREAM_SELECT  |PARTITIONED|
-                                assign [$$94, $$91, $$93] <- [$$ht.getField("phone"), $$ht.getField("reviews"), $$ht.getField("name")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                -- ASSIGN  |PARTITIONED|
-                                  project ([$$ht]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      data-scan []<-[$$85, $$ht] <- test.ColumnDataset project ({reviews:[{ratings:{Overall:any}}],phone:any,city:any,name:any}) filter on: eq($$ht.getField("city"), "Los Angeles") range-filter on: eq($$ht.getField("city"), "Los Angeles") [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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: 2.0, op-cost: 0.0, total-cost: 2.0]
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                data-scan []<-[$$85, $$ht] <- test.ColumnDataset project ({reviews:[{ratings:{Overall:any}}],phone:any,city:any,name:any}) filter on: eq($$ht.getField("city"), "Los Angeles") range-filter on: eq($$ht.getField("city"), "Los Angeles") [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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_cbo/column/pushdown/other-pushdowns/other-pushdowns.023.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.023.plan
index f685929..59f8b47 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.023.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.023.plan
@@ -6,23 +6,19 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        project ([$$19]) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          assign [$$19] <- [{"payload": $$20.getField("payload")}] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-          -- ASSIGN  |PARTITIONED|
-            limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-            -- STREAM_LIMIT  |PARTITIONED|
-              project ([$$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+        assign [$$19] <- [{"payload": $$20.getField("payload")}] project: [$$19] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+        -- ASSIGN  |PARTITIONED|
+          limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+          -- STREAM_LIMIT  |PARTITIONED|
+            assign [$$20] <- [$$o.getField("v")] project: [$$20] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+            -- ASSIGN  |PARTITIONED|
+              project ([$$o]) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
               -- STREAM_PROJECT  |PARTITIONED|
-                assign [$$20] <- [$$o.getField("v")] [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-                -- ASSIGN  |PARTITIONED|
-                  project ([$$o]) [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 2.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$21, $$o] <- test.ColumnDataset condition (eq($$o.getField("v").getField("type"), "WeMo")) limit 10 project ({v:{payload:any,type:any}}) filter on: eq($$o.getField("v").getField("type"), "WeMo") range-filter on: eq($$o.getField("v").getField("type"), "WeMo") [cardinality: 0.0, op-cost: 2.0, total-cost: 2.0]
+                  -- DATASOURCE_SCAN  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      data-scan []<-[$$21, $$o] <- test.ColumnDataset condition (eq($$o.getField("v").getField("type"), "WeMo")) limit 10 project ({v:{payload:any,type:any}}) filter on: eq($$o.getField("v").getField("type"), "WeMo") range-filter on: eq($$o.getField("v").getField("type"), "WeMo") [cardinality: 0.0, op-cost: 2.0, total-cost: 2.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|
+                      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_cbo/column/pushdown/other-pushdowns/other-pushdowns.024.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.024.plan
index fde7f81..37d3a8d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.024.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.024.plan
@@ -2,37 +2,29 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$31]) [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$31] <- [{"$1": get-item(get-item($$36, 0), 0), "$2": get-item($$37, 1), "$3": get-item(get-item($$38, 1), 1)}] [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$36, $$37, $$38]) [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-          -- SORT_MERGE_EXCHANGE [$$35(ASC) ]  |PARTITIONED|
-            order (ASC, $$35) [cardinality: 2.0, op-cost: 2.0, total-cost: 4.0]
-            -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
-              exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$36, $$37, $$38, $$35]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$38, $$37] <- [$$49.getField("f2"), $$49.getField("f3")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+    assign [$$31] <- [{"$1": get-item(get-item($$36, 0), 0), "$2": get-item($$37, 1), "$3": get-item(get-item($$38, 1), 1)}] project: [$$31] [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$36, $$37, $$38]) [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
+        -- SORT_MERGE_EXCHANGE [$$35(ASC) ]  |PARTITIONED|
+          order (ASC, $$35) [cardinality: 2.0, op-cost: 2.0, total-cost: 4.0]
+          -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+            exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$38, $$37] <- [$$49.getField("f2"), $$49.getField("f3")] project: [$$36, $$37, $$38, $$35] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+              -- ASSIGN  |PARTITIONED|
+                assign [$$49, $$36] <- [get-item($$33, 1), get-item($$33, 0).getField("f2")] project: [$$35, $$49, $$36] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                -- ASSIGN  |PARTITIONED|
+                  assign [$$33, $$35] <- [$$c.getField("f1"), $$c.getField("x")] project: [$$33, $$35] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                   -- ASSIGN  |PARTITIONED|
-                    project ([$$35, $$49, $$36]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                    project ([$$c]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                     -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$49, $$36] <- [get-item($$33, 1), get-item($$33, 0).getField("f2")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                      -- ASSIGN  |PARTITIONED|
-                        project ([$$33, $$35]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$33, $$35] <- [$$c.getField("f1"), $$c.getField("x")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                          -- ASSIGN  |PARTITIONED|
-                            project ([$$c]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                data-scan []<-[$$34, $$c] <- test.ColumnDataset project ({x:any,f1:[{f2:[[any]],f3:[any]}]}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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: 2.0, op-cost: 0.0, total-cost: 2.0]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        data-scan []<-[$$34, $$c] <- test.ColumnDataset project ({x:any,f1:[{f2:[[any]],f3:[any]}]}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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_cbo/column/pushdown/other-pushdowns/other-pushdowns.025.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.025.plan
index deeb1eb..d8a8e6a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.025.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.025.plan
@@ -2,21 +2,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$32]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$32] <- [{"$1": get-item(get-item(get-item($$33, 0).getField("f2"), 0), 0), "$2": get-item($$49.getField("f3"), 1), "$3": get-item(get-item($$49.getField("f2"), 1), 1), "x": $$c.getField("x")}] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+    assign [$$32] <- [{"$1": get-item(get-item(get-item($$33, 0).getField("f2"), 0), 0), "$2": get-item($$49.getField("f3"), 1), "$3": get-item(get-item($$49.getField("f2"), 1), 1), "x": $$c.getField("x")}] project: [$$32] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+    -- ASSIGN  |PARTITIONED|
+      assign [$$49] <- [get-item($$33, 1)] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
       -- ASSIGN  |PARTITIONED|
-        assign [$$49] <- [get-item($$33, 1)] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+        assign [$$33] <- [$$c.getField("f1")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
         -- ASSIGN  |PARTITIONED|
-          assign [$$33] <- [$$c.getField("f1")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-          -- ASSIGN  |PARTITIONED|
-            project ([$$c]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                data-scan []<-[$$34, $$c] <- test.ColumnDataset project ({x:any,f1:[{f2:[[any]],f3:[any]}]}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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 ([$$c]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+          -- STREAM_PROJECT  |PARTITIONED|
+            exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$34, $$c] <- test.ColumnDataset project ({x:any,f1:[{f2:[[any]],f3:[any]}]}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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_cbo/column/pushdown/other-pushdowns/other-pushdowns.026.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.026.plan
index 2379a5a..2ecaed1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.026.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.026.plan
@@ -2,37 +2,29 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$32]) [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$32] <- [{"$1": get-item(get-item($$37, 0), 0), "$2": get-item($$38, 1), "$3": get-item(get-item($$39, 1), 1), "x": $$40}] [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$37, $$38, $$39, $$40]) [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-          -- SORT_MERGE_EXCHANGE [$$36(ASC) ]  |PARTITIONED|
-            order (ASC, $$36) [cardinality: 2.0, op-cost: 2.0, total-cost: 4.0]
-            -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
-              exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$37, $$38, $$39, $$40, $$36]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$39, $$38] <- [$$51.getField("f2"), $$51.getField("f3")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+    assign [$$32] <- [{"$1": get-item(get-item($$37, 0), 0), "$2": get-item($$38, 1), "$3": get-item(get-item($$39, 1), 1), "x": $$40}] project: [$$32] [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$37, $$38, $$39, $$40]) [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
+        -- SORT_MERGE_EXCHANGE [$$36(ASC) ]  |PARTITIONED|
+          order (ASC, $$36) [cardinality: 2.0, op-cost: 2.0, total-cost: 4.0]
+          -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
+            exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              assign [$$39, $$38] <- [$$51.getField("f2"), $$51.getField("f3")] project: [$$37, $$38, $$39, $$40, $$36] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+              -- ASSIGN  |PARTITIONED|
+                assign [$$51, $$37] <- [get-item($$34, 1), get-item($$34, 0).getField("f2")] project: [$$36, $$40, $$51, $$37] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                -- ASSIGN  |PARTITIONED|
+                  assign [$$34, $$36, $$40] <- [$$c.getField("f1"), $$c.getField("y"), $$c.getField("x")] project: [$$34, $$36, $$40] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                   -- ASSIGN  |PARTITIONED|
-                    project ([$$36, $$40, $$51, $$37]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                    project ([$$c]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                     -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$51, $$37] <- [get-item($$34, 1), get-item($$34, 0).getField("f2")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                      -- ASSIGN  |PARTITIONED|
-                        project ([$$34, $$36, $$40]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$34, $$36, $$40] <- [$$c.getField("f1"), $$c.getField("y"), $$c.getField("x")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                          -- ASSIGN  |PARTITIONED|
-                            project ([$$c]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                data-scan []<-[$$35, $$c] <- test.ColumnDataset project ({x:any,y:any,f1:[{f2:[[any]],f3:[any]}]}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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: 2.0, op-cost: 0.0, total-cost: 2.0]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        data-scan []<-[$$35, $$c] <- test.ColumnDataset project ({x:any,y:any,f1:[{f2:[[any]],f3:[any]}]}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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_cbo/column/pushdown/other-pushdowns/other-pushdowns.027.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.027.plan
index 66b024a..122a411 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.027.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/column/pushdown/other-pushdowns/other-pushdowns.027.plan
@@ -2,25 +2,21 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$35]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$35] <- [{"$1": get-item(get-item(get-item($$36, 0).getField("f2"), 0), 0), "$2": get-item($$43, 1), "$3": get-item(get-item($$55.getField("f2"), 1), 1), "f4": $$43.getField("f4")}] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+    assign [$$35] <- [{"$1": get-item(get-item(get-item($$36, 0).getField("f2"), 0), 0), "$2": get-item($$43, 1), "$3": get-item(get-item($$55.getField("f2"), 1), 1), "f4": $$43.getField("f4")}] project: [$$35] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+    -- ASSIGN  |PARTITIONED|
+      assign [$$43] <- [$$55.getField("f3")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
       -- ASSIGN  |PARTITIONED|
-        assign [$$43] <- [$$55.getField("f3")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+        assign [$$55] <- [get-item($$36, 1)] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
         -- ASSIGN  |PARTITIONED|
-          assign [$$55] <- [get-item($$36, 1)] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+          assign [$$36] <- [$$c.getField("f1")] project: [$$36] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
           -- ASSIGN  |PARTITIONED|
-            project ([$$36]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+            project ([$$c]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
             -- STREAM_PROJECT  |PARTITIONED|
-              assign [$$36] <- [$$c.getField("f1")] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-              -- ASSIGN  |PARTITIONED|
-                project ([$$c]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+              exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                data-scan []<-[$$37, $$c] <- test.ColumnDataset project ({f1:[{f2:[[any]],f3:<[any],{f4:any}>}]}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.0]
+                -- DATASOURCE_SCAN  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$37, $$c] <- test.ColumnDataset project ({f1:[{f2:[[any]],f3:<[any],{f4:any}>}]}) [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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|
+                    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_cbo/explain/explain_field_access/explain_field_access.1.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/explain/explain_field_access/explain_field_access.1.plan
index fb83a56..b44bf46 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/explain/explain_field_access/explain_field_access.1.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/explain/explain_field_access/explain_field_access.1.plan
@@ -2,41 +2,37 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$50]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$50] <- [{"deptId": $#1, "star_cost": $$53}] [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]
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          group by ([$#1 := $$58]) decor ([]) {
-                    aggregate [$$53] <- [agg-global-sql-sum($$57)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$58]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-              group by ([$$58 := $$51]) decor ([]) {
-                        aggregate [$$57] <- [agg-local-sql-sum($$48)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$51]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$48, $$51]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$50] <- [{"deptId": $#1, "star_cost": $$53}] project: [$$50] [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]
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        group by ([$#1 := $$58]) decor ([]) {
+                  aggregate [$$53] <- [agg-global-sql-sum($$57)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$58]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
+            group by ([$$58 := $$51]) decor ([]) {
+                      aggregate [$$57] <- [agg-local-sql-sum($$48)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$51]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                assign [$$51, $$48] <- [substring($$e.getField("dept").getField("department_id"), 0), $$e.getField("salary")] project: [$$48, $$51] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$e]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$51, $$48] <- [substring($$e.getField("dept").getField("department_id"), 0), $$e.getField("salary")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
-                      project ([$$e]) [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 []<-[$$52, $$e] <- gby.Employee [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|
-                          data-scan []<-[$$52, $$e] <- gby.Employee [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|
+                          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_cbo/explain/explain_field_access_closed/explain_field_access_closed.1.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/explain/explain_field_access_closed/explain_field_access_closed.1.plan
index bd96df9..d8a5148 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/explain/explain_field_access_closed/explain_field_access_closed.1.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/explain/explain_field_access_closed/explain_field_access_closed.1.plan
@@ -2,41 +2,37 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$49]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$49] <- [{"deptId": $#1, "star_cost": $$52}] [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]
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          group by ([$#1 := $$56]) decor ([]) {
-                    aggregate [$$52] <- [agg-global-sql-sum($$55)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$56]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
-              group by ([$$56 := $$50]) decor ([]) {
-                        aggregate [$$55] <- [agg-local-sql-sum($$47)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$50]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$47, $$50]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$49] <- [{"deptId": $#1, "star_cost": $$52}] project: [$$49] [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]
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        group by ([$#1 := $$56]) decor ([]) {
+                  aggregate [$$52] <- [agg-global-sql-sum($$55)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$56]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
+            group by ([$$56 := $$50]) decor ([]) {
+                      aggregate [$$55] <- [agg-local-sql-sum($$47)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$50]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                assign [$$50, $$47] <- [substring($$e.getField(1), 0), $$e.getField(2)] project: [$$47, $$50] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$e]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$50, $$47] <- [substring($$e.getField(1), 0), $$e.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
-                      project ([$$e]) [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 []<-[$$51, $$e] <- gby.Employee [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|
-                          data-scan []<-[$$51, $$e] <- gby.Employee [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|
+                          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_cbo/index-selection/like-expression/like-expression.04.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/index-selection/like-expression/like-expression.04.plan
new file mode 100644
index 0000000..3a398d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/index-selection/like-expression/like-expression.04.plan
@@ -0,0 +1,34 @@
+distribute result [$$19] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    assign [$$19] <- [{"ds1": $$ds1}] project: [$$19] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+    -- ASSIGN  |PARTITIONED|
+      select (and(ge($$20, "fsdxv😀"), lt($$20, "fsdxv😁"))) project: [$$ds1] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        assign [$$20] <- [$$ds1.getField(1)] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+        -- ASSIGN  |PARTITIONED|
+          project ([$$ds1]) [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+          -- STREAM_PROJECT  |PARTITIONED|
+            exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              unnest-map [$$21, $$ds1] <- index-search("ds1", 0, "Default", "test", "ds1", false, false, 1, $$25, 1, $$25, true, true, true) [cardinality: 1.0, op-cost: 2.0, total-cost: 2.0]
+              -- BTREE_SEARCH  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$25) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      project ([$$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 [$$24, $$25] <- index-search("ds1_name", 0, "Default", "test", "ds1", false, false, 1, $$22, 1, $$23, 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 [$$22, $$23] <- ["fsdxv😀", "fsdxv😁"] [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_cbo/index-selection/like-expression/like-expression.06.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/index-selection/like-expression/like-expression.06.plan
new file mode 100644
index 0000000..cee7f08
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/index-selection/like-expression/like-expression.06.plan
@@ -0,0 +1,34 @@
+distribute result [$$19] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    assign [$$19] <- [{"ds1": $$ds1}] project: [$$19] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+    -- ASSIGN  |PARTITIONED|
+      select (and(ge($$20, "asdgdh👩‍👩‍👧"), lt($$20, "asdgdh👩‍👩‍👨"))) project: [$$ds1] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        assign [$$20] <- [$$ds1.getField(1)] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+        -- ASSIGN  |PARTITIONED|
+          project ([$$ds1]) [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+          -- STREAM_PROJECT  |PARTITIONED|
+            exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              unnest-map [$$21, $$ds1] <- index-search("ds1", 0, "Default", "test", "ds1", false, false, 1, $$25, 1, $$25, true, true, true) [cardinality: 1.0, op-cost: 2.0, total-cost: 2.0]
+              -- BTREE_SEARCH  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$25) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      project ([$$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 [$$24, $$25] <- index-search("ds1_name", 0, "Default", "test", "ds1", false, false, 1, $$22, 1, $$23, 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 [$$22, $$23] <- ["asdgdh👩‍👩‍👧", "asdgdh👩‍👩‍👨"] [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_cbo/index-selection/like-expression/like-expression.08.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/index-selection/like-expression/like-expression.08.plan
new file mode 100644
index 0000000..a5d1097
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/index-selection/like-expression/like-expression.08.plan
@@ -0,0 +1,32 @@
+distribute result [$$16] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    assign [$$16] <- [{"ds1": $$ds1}] project: [$$16] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+    -- ASSIGN  |PARTITIONED|
+      select (eq($$ds1.getField(1), "sxvciis")) [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$ds1]) [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+        -- STREAM_PROJECT  |PARTITIONED|
+          exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            unnest-map [$$17, $$ds1] <- index-search("ds1", 0, "Default", "test", "ds1", false, false, 1, $$22, 1, $$22, true, true, true) [cardinality: 1.0, op-cost: 2.0, total-cost: 2.0]
+            -- BTREE_SEARCH  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                order (ASC, $$22) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    project ([$$22]) [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] <- index-search("ds1_name", 0, "Default", "test", "ds1", false, false, 1, $$19, 1, $$20, true, true, true) [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, $$20] <- ["sxvciis", "sxvciis"] [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_cbo/index-selection/like-expression/like-expression.10.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/index-selection/like-expression/like-expression.10.plan
new file mode 100644
index 0000000..a2ad9f5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/index-selection/like-expression/like-expression.10.plan
@@ -0,0 +1,34 @@
+distribute result [$$19] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    assign [$$19] <- [{"ds1": $$ds1}] project: [$$19] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+    -- ASSIGN  |PARTITIONED|
+      select (and(ge($$20, "Joh"), lt($$20, "Joi"))) project: [$$ds1] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        assign [$$20] <- [$$ds1.getField(1)] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+        -- ASSIGN  |PARTITIONED|
+          project ([$$ds1]) [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+          -- STREAM_PROJECT  |PARTITIONED|
+            exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              unnest-map [$$21, $$ds1] <- index-search("ds1", 0, "Default", "test", "ds1", false, false, 1, $$25, 1, $$25, true, true, true) [cardinality: 1.0, op-cost: 2.0, total-cost: 2.0]
+              -- BTREE_SEARCH  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$25) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      project ([$$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 [$$24, $$25] <- index-search("ds1_name", 0, "Default", "test", "ds1", false, false, 1, $$22, 1, $$23, 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 [$$22, $$23] <- ["Joh", "Joi"] [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_cbo/index-selection/like-expression/like-expression.12.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/index-selection/like-expression/like-expression.12.plan
new file mode 100644
index 0000000..70446b6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/index-selection/like-expression/like-expression.12.plan
@@ -0,0 +1,18 @@
+distribute result [$$16] [cardinality: 1.0, op-cost: 0.0, total-cost: 19.0]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 19.0]
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    assign [$$16] <- [{"ds1": $$ds1}] project: [$$16] [cardinality: 1.0, op-cost: 0.0, total-cost: 19.0]
+    -- ASSIGN  |PARTITIONED|
+      select (like($$ds1.getField(1), "s%vfjs__jiscii%")) [cardinality: 1.0, op-cost: 0.0, total-cost: 19.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$ds1]) [cardinality: 19.0, op-cost: 0.0, total-cost: 19.0]
+        -- STREAM_PROJECT  |PARTITIONED|
+          exchange [cardinality: 19.0, op-cost: 0.0, total-cost: 19.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            data-scan []<-[$$17, $$ds1] <- test.ds1 [cardinality: 19.0, op-cost: 19.0, total-cost: 19.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_cbo/index-selection/like-expression/like-expression.14.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/index-selection/like-expression/like-expression.14.plan
new file mode 100644
index 0000000..f2386ee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/index-selection/like-expression/like-expression.14.plan
@@ -0,0 +1,18 @@
+distribute result [$$16] [cardinality: 1.0, op-cost: 0.0, total-cost: 19.0]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 19.0]
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    assign [$$16] <- [{"ds1": $$ds1}] project: [$$16] [cardinality: 1.0, op-cost: 0.0, total-cost: 19.0]
+    -- ASSIGN  |PARTITIONED|
+      select (like($$ds1.getField(1), "sxvc_is")) [cardinality: 1.0, op-cost: 0.0, total-cost: 19.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$ds1]) [cardinality: 19.0, op-cost: 0.0, total-cost: 19.0]
+        -- STREAM_PROJECT  |PARTITIONED|
+          exchange [cardinality: 19.0, op-cost: 0.0, total-cost: 19.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            data-scan []<-[$$17, $$ds1] <- test.ds1 [cardinality: 19.0, op-cost: 19.0, total-cost: 19.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_cbo/index-selection/like-expression/like-expression.16.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/index-selection/like-expression/like-expression.16.plan
new file mode 100644
index 0000000..abd456d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/index-selection/like-expression/like-expression.16.plan
@@ -0,0 +1,18 @@
+distribute result [$$16] [cardinality: 1.0, op-cost: 0.0, total-cost: 19.0]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 19.0]
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    assign [$$16] <- [{"ds1": $$ds1}] project: [$$16] [cardinality: 1.0, op-cost: 0.0, total-cost: 19.0]
+    -- ASSIGN  |PARTITIONED|
+      select (like($$ds1.getField(1), "sxvc_i%")) [cardinality: 1.0, op-cost: 0.0, total-cost: 19.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$ds1]) [cardinality: 19.0, op-cost: 0.0, total-cost: 19.0]
+        -- STREAM_PROJECT  |PARTITIONED|
+          exchange [cardinality: 19.0, op-cost: 0.0, total-cost: 19.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            data-scan []<-[$$17, $$ds1] <- test.ds1 [cardinality: 19.0, op-cost: 19.0, total-cost: 19.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_cbo/index-selection/like-expression/like-expression.18.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/index-selection/like-expression/like-expression.18.plan
new file mode 100644
index 0000000..091ce85
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/index-selection/like-expression/like-expression.18.plan
@@ -0,0 +1,18 @@
+distribute result [$$16] [cardinality: 1.0, op-cost: 0.0, total-cost: 19.0]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 19.0]
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    assign [$$16] <- [{"ds1": $$ds1}] project: [$$16] [cardinality: 1.0, op-cost: 0.0, total-cost: 19.0]
+    -- ASSIGN  |PARTITIONED|
+      select (like($$ds1.getField(1), "sxvfjs%jisci%")) [cardinality: 1.0, op-cost: 0.0, total-cost: 19.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$ds1]) [cardinality: 19.0, op-cost: 0.0, total-cost: 19.0]
+        -- STREAM_PROJECT  |PARTITIONED|
+          exchange [cardinality: 19.0, op-cost: 0.0, total-cost: 19.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            data-scan []<-[$$17, $$ds1] <- test.ds1 [cardinality: 19.0, op-cost: 19.0, total-cost: 19.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_cbo/index-selection/like-expression/like-expression.20.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/index-selection/like-expression/like-expression.20.plan
new file mode 100644
index 0000000..24e1def
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/index-selection/like-expression/like-expression.20.plan
@@ -0,0 +1,20 @@
+distribute result [$$19] [cardinality: 2.0, op-cost: 0.0, total-cost: 19.0]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 19.0]
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    assign [$$19] <- [{"ds1": $$ds1}] project: [$$19] [cardinality: 2.0, op-cost: 0.0, total-cost: 19.0]
+    -- ASSIGN  |PARTITIONED|
+      select (and(ge($$20, "J"), lt($$20, "K"))) project: [$$ds1] [cardinality: 2.0, op-cost: 0.0, total-cost: 19.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        assign [$$20] <- [$$ds1.getField(1)] [cardinality: 19.0, op-cost: 0.0, total-cost: 19.0]
+        -- ASSIGN  |PARTITIONED|
+          project ([$$ds1]) [cardinality: 19.0, op-cost: 0.0, total-cost: 19.0]
+          -- STREAM_PROJECT  |PARTITIONED|
+            exchange [cardinality: 19.0, op-cost: 0.0, total-cost: 19.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$21, $$ds1] <- test.ds1 [cardinality: 19.0, op-cost: 19.0, total-cost: 19.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_cbo/index-selection/like-expression/like-expression.22.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/index-selection/like-expression/like-expression.22.plan
new file mode 100644
index 0000000..d9ac312
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/index-selection/like-expression/like-expression.22.plan
@@ -0,0 +1,20 @@
+distribute result [$$19] [cardinality: 2.0, op-cost: 0.0, total-cost: 19.0]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 19.0]
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    assign [$$19] <- [{"ds1": $$ds1}] project: [$$19] [cardinality: 2.0, op-cost: 0.0, total-cost: 19.0]
+    -- ASSIGN  |PARTITIONED|
+      select (and(ge($$20, "sx"), lt($$20, "sy"))) project: [$$ds1] [cardinality: 2.0, op-cost: 0.0, total-cost: 19.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        assign [$$20] <- [$$ds1.getField(1)] [cardinality: 19.0, op-cost: 0.0, total-cost: 19.0]
+        -- ASSIGN  |PARTITIONED|
+          project ([$$ds1]) [cardinality: 19.0, op-cost: 0.0, total-cost: 19.0]
+          -- STREAM_PROJECT  |PARTITIONED|
+            exchange [cardinality: 19.0, op-cost: 0.0, total-cost: 19.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$21, $$ds1] <- test.ds1 [cardinality: 19.0, op-cost: 19.0, total-cost: 19.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_cbo/index-selection/like-expression/like-expression.25.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/index-selection/like-expression/like-expression.25.plan
new file mode 100644
index 0000000..5b0186e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/index-selection/like-expression/like-expression.25.plan
@@ -0,0 +1,32 @@
+distribute result [$$16] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    assign [$$16] <- [{"ds1": $$ds1}] project: [$$16] [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+    -- ASSIGN  |PARTITIONED|
+      select (eq($$ds1.getField(1), "h\axx%")) [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$ds1]) [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+        -- STREAM_PROJECT  |PARTITIONED|
+          exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 2.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            unnest-map [$$17, $$ds1] <- index-search("ds1", 0, "Default", "test", "ds1", false, false, 1, $$22, 1, $$22, true, true, true) [cardinality: 1.0, op-cost: 2.0, total-cost: 2.0]
+            -- BTREE_SEARCH  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                order (ASC, $$22) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    project ([$$22]) [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] <- index-search("ds1_name", 0, "Default", "test", "ds1", false, false, 1, $$19, 1, $$20, true, true, true) [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, $$20] <- ["h\axx%", "h\axx%"] [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_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.04.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.04.plan
index f4c4a85..b5e1560 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.04.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.04.plan
@@ -1,64 +1,58 @@
-distribute result [$$51] [cardinality: 159.84, op-cost: 0.0, total-cost: 1777.06]
+distribute result [$$51] [cardinality: 166.67, op-cost: 0.0, total-cost: 1840.73]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 159.84, op-cost: 0.0, total-cost: 1777.06]
+  exchange [cardinality: 166.67, op-cost: 0.0, total-cost: 1840.73]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$51]) [cardinality: 159.84, op-cost: 0.0, total-cost: 1777.06]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$51] <- [{"n_nationkey": $$58, "s_nationkey": $$56, "c_nationkey": $$55}] [cardinality: 159.84, op-cost: 0.0, total-cost: 1777.06]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 159.84, op-cost: 0.0, total-cost: 1777.06]
-        -- SORT_MERGE_EXCHANGE [$$58(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
-          order (ASC, $$58) (ASC, $$56) (ASC, $$55) [cardinality: 159.84, op-cost: 1170.11, total-cost: 1777.06]
-          -- STABLE_SORT [$$58(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
-            exchange [cardinality: 159.84, op-cost: 0.0, total-cost: 606.95]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              join (eq($$55, $$58)) [cardinality: 159.84, op-cost: 175.97, total-cost: 606.95]
-              -- HYBRID_HASH_JOIN [$$55][$$58]  |PARTITIONED|
-                exchange [cardinality: 150.0, op-cost: 150.0, total-cost: 300.0]
-                -- HASH_PARTITION_EXCHANGE [$$55]  |PARTITIONED|
-                  project ([$$55]) [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
+    assign [$$51] <- [{"n_nationkey": $$58, "s_nationkey": $$56, "c_nationkey": $$55}] project: [$$51] [cardinality: 166.67, op-cost: 0.0, total-cost: 1840.73]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 166.67, op-cost: 0.0, total-cost: 1840.73]
+      -- SORT_MERGE_EXCHANGE [$$58(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
+        order (ASC, $$58) (ASC, $$56) (ASC, $$55) [cardinality: 166.67, op-cost: 1230.17, total-cost: 1840.73]
+        -- STABLE_SORT [$$58(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
+          exchange [cardinality: 166.67, op-cost: 0.0, total-cost: 610.56]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            join (eq($$55, $$58)) [cardinality: 166.67, op-cost: 177.78, total-cost: 610.56]
+            -- HYBRID_HASH_JOIN [$$55][$$58]  |PARTITIONED|
+              exchange [cardinality: 150.0, op-cost: 150.0, total-cost: 300.0]
+              -- HASH_PARTITION_EXCHANGE [$$55]  |PARTITIONED|
+                assign [$$55] <- [$$c.getField(3)] project: [$$55] [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$c]) [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$55] <- [$$c.getField(3)] [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
+                    exchange [cardinality: 150.0, op-cost: 150.0, total-cost: 300.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$60, $$c] <- tpch.Customer [cardinality: 150.0, op-cost: 150.0, total-cost: 150.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: 27.78, op-cost: 27.78, total-cost: 132.78]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                join (eq($$56, $$58)) [cardinality: 27.78, op-cost: 35.0, total-cost: 105.0]
+                -- HYBRID_HASH_JOIN [$$58][$$56]  |PARTITIONED|
+                  exchange [cardinality: 25.0, op-cost: 25.0, total-cost: 50.0]
+                  -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
+                    project ([$$58]) [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      exchange [cardinality: 25.0, op-cost: 25.0, total-cost: 50.0]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        data-scan []<-[$$58, $$n] <- tpch.Nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.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: 10.0, op-cost: 10.0, total-cost: 20.0]
+                  -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
+                    assign [$$56] <- [$$s.getField(3)] project: [$$56] [cardinality: 10.0, op-cost: 0.0, total-cost: 10.0]
                     -- ASSIGN  |PARTITIONED|
-                      project ([$$c]) [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
+                      project ([$$s]) [cardinality: 10.0, op-cost: 0.0, total-cost: 10.0]
                       -- STREAM_PROJECT  |PARTITIONED|
-                        exchange [cardinality: 150.0, op-cost: 150.0, total-cost: 300.0]
+                        exchange [cardinality: 10.0, op-cost: 10.0, total-cost: 20.0]
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$60, $$c] <- tpch.Customer [cardinality: 150.0, op-cost: 150.0, total-cost: 150.0]
+                          data-scan []<-[$$59, $$s] <- tpch.Supplier [cardinality: 10.0, op-cost: 10.0, total-cost: 10.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: 25.97, op-cost: 25.97, total-cost: 130.97]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  join (eq($$56, $$58)) [cardinality: 25.97, op-cost: 35.0, total-cost: 105.0]
-                  -- HYBRID_HASH_JOIN [$$58][$$56]  |PARTITIONED|
-                    exchange [cardinality: 25.0, op-cost: 25.0, total-cost: 50.0]
-                    -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-                      project ([$$58]) [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        exchange [cardinality: 25.0, op-cost: 25.0, total-cost: 50.0]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$58, $$n] <- tpch.Nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.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: 10.0, op-cost: 10.0, total-cost: 20.0]
-                    -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
-                      project ([$$56]) [cardinality: 10.0, op-cost: 0.0, total-cost: 10.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$56] <- [$$s.getField(3)] [cardinality: 10.0, op-cost: 0.0, total-cost: 10.0]
-                        -- ASSIGN  |PARTITIONED|
-                          project ([$$s]) [cardinality: 10.0, op-cost: 0.0, total-cost: 10.0]
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            exchange [cardinality: 10.0, op-cost: 10.0, total-cost: 20.0]
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              data-scan []<-[$$59, $$s] <- tpch.Supplier [cardinality: 10.0, op-cost: 10.0, total-cost: 10.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_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.06.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.06.plan
index 668a8ec..d57b4fd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.06.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.06.plan
@@ -2,37 +2,33 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 6016.3, op-cost: 0.0, total-cost: 95054.55]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$36]) [cardinality: 6016.3, op-cost: 0.0, total-cost: 95054.55]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$36] <- [{"o_orderkey": $$43, "l_orderkey": $$44, "l_suppkey": $$42}] [cardinality: 6016.3, op-cost: 0.0, total-cost: 95054.55]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 6016.3, op-cost: 0.0, total-cost: 95054.55]
-        -- SORT_MERGE_EXCHANGE [$$43(ASC), $$44(ASC), $$42(ASC) ]  |PARTITIONED|
-          order (ASC, $$43) (ASC, $$44) (ASC, $$42) [cardinality: 6016.3, op-cost: 75532.61, total-cost: 95054.55]
-          -- STABLE_SORT [$$43(ASC), $$44(ASC), $$42(ASC)]  |PARTITIONED|
-            exchange [cardinality: 6016.3, op-cost: 0.0, total-cost: 19521.94]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              select (eq($$43, $$42)) [cardinality: 6016.3, op-cost: 12016.29, total-cost: 19521.94]
-              -- STREAM_SELECT  |PARTITIONED|
-                project ([$$43, $$44, $$42]) [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
+    assign [$$36] <- [{"o_orderkey": $$43, "l_orderkey": $$44, "l_suppkey": $$42}] project: [$$36] [cardinality: 6016.3, op-cost: 0.0, total-cost: 95054.55]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 6016.3, op-cost: 0.0, total-cost: 95054.55]
+      -- SORT_MERGE_EXCHANGE [$$43(ASC), $$44(ASC), $$42(ASC) ]  |PARTITIONED|
+        order (ASC, $$43) (ASC, $$44) (ASC, $$42) [cardinality: 6016.3, op-cost: 75532.61, total-cost: 95054.55]
+        -- STABLE_SORT [$$43(ASC), $$44(ASC), $$42(ASC)]  |PARTITIONED|
+          exchange [cardinality: 6016.3, op-cost: 0.0, total-cost: 19521.94]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select (eq($$43, $$42)) [cardinality: 6016.3, op-cost: 12016.29, total-cost: 19521.94]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$42] <- [$$l.getField(2)] project: [$$43, $$44, $$42] [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
+              -- ASSIGN  |PARTITIONED|
+                project ([$$43, $$44, $$l]) [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
                 -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$42] <- [$$l.getField(2)] [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
-                  -- ASSIGN  |PARTITIONED|
-                    project ([$$43, $$44, $$l]) [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      exchange [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        unnest-map [$$44, $$45, $$l] <- index-search("LineItem", 0, "Default", "tpch", "LineItem", true, true, 1, $$43, 1, $$43, true, true, true) [cardinality: 6010.65, op-cost: 6005.0, total-cost: 6005.0]
-                        -- BTREE_SEARCH  |PARTITIONED|
+                  exchange [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    unnest-map [$$44, $$45, $$l] <- index-search("LineItem", 0, "Default", "tpch", "LineItem", true, true, 1, $$43, 1, $$43, true, true, true) [cardinality: 6010.65, op-cost: 6005.0, total-cost: 6005.0]
+                    -- BTREE_SEARCH  |PARTITIONED|
+                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        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]
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            project ([$$43]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            data-scan []<-[$$43, $$o] <- tpch.Orders [cardinality: 1500.0, op-cost: 1500.0, total-cost: 1500.0]
+                            -- DATASOURCE_SCAN  |PARTITIONED|
                               exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                data-scan []<-[$$43, $$o] <- tpch.Orders [cardinality: 1500.0, op-cost: 1500.0, total-cost: 1500.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|
+                                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_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.08.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.08.plan
index 0668ea4..866663f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.08.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.08.plan
@@ -2,49 +2,45 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 1500.0, op-cost: 0.0, total-cost: 77032.61]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$38]) [cardinality: 1500.0, op-cost: 0.0, total-cost: 77032.61]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$38] <- [{"o_orderkey": $$43, "l_orderkey": $$44, "l_suppkey": $$47}] [cardinality: 1500.0, op-cost: 0.0, total-cost: 77032.61]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 1500.0, op-cost: 0.0, total-cost: 77032.61]
-        -- SORT_MERGE_EXCHANGE [$$43(ASC), $$44(ASC), $$47(ASC) ]  |PARTITIONED|
-          order (ASC, $$43) (ASC, $$44) (ASC, $$47) [cardinality: 1500.0, op-cost: 75532.61, total-cost: 77032.61]
-          -- STABLE_SORT [$$43(ASC), $$44(ASC), $$47(ASC)]  |PARTITIONED|
-            exchange [cardinality: 1500.0, op-cost: 0.0, total-cost: 1500.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$43, $$44, $$47]) [cardinality: 1500.0, op-cost: 0.0, total-cost: 1500.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                exchange [cardinality: 1500.0, op-cost: 0.0, total-cost: 1500.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  left outer join (and(eq($$43, $$44), eq($$56, $$47))) [cardinality: 6016.3, op-cost: 7512.06, total-cost: 22529.12]
-                  -- HYBRID_HASH_JOIN [$$43, $$56][$$44, $$47]  |PARTITIONED|
-                    exchange [cardinality: 1500.0, op-cost: 0.0, total-cost: 1500.0]
-                    -- HASH_PARTITION_EXCHANGE [$$43, $$56]  |PARTITIONED|
-                      assign [$$56] <- [$$43] [cardinality: 1500.0, op-cost: 0.0, total-cost: 1500.0]
-                      -- ASSIGN  |PARTITIONED|
-                        project ([$$43]) [cardinality: 1500.0, op-cost: 0.0, total-cost: 1500.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          exchange [cardinality: 1500.0, op-cost: 0.0, total-cost: 1500.0]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            data-scan []<-[$$43, $$o] <- tpch.Orders [cardinality: 1500.0, op-cost: 1500.0, total-cost: 1500.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 [$$44, $$47]  |PARTITIONED|
-                      project ([$$44, $$47]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$38] <- [{"o_orderkey": $$43, "l_orderkey": $$44, "l_suppkey": $$47}] project: [$$38] [cardinality: 1500.0, op-cost: 0.0, total-cost: 77032.61]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 1500.0, op-cost: 0.0, total-cost: 77032.61]
+      -- SORT_MERGE_EXCHANGE [$$43(ASC), $$44(ASC), $$47(ASC) ]  |PARTITIONED|
+        order (ASC, $$43) (ASC, $$44) (ASC, $$47) [cardinality: 1500.0, op-cost: 75532.61, total-cost: 77032.61]
+        -- STABLE_SORT [$$43(ASC), $$44(ASC), $$47(ASC)]  |PARTITIONED|
+          exchange [cardinality: 1500.0, op-cost: 0.0, total-cost: 1500.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            project ([$$43, $$44, $$47]) [cardinality: 1500.0, op-cost: 0.0, total-cost: 1500.0]
+            -- STREAM_PROJECT  |PARTITIONED|
+              exchange [cardinality: 1500.0, op-cost: 0.0, total-cost: 1500.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                left outer join (and(eq($$43, $$44), eq($$56, $$47))) [cardinality: 6016.3, op-cost: 7512.06, total-cost: 22529.12]
+                -- HYBRID_HASH_JOIN [$$43, $$56][$$44, $$47]  |PARTITIONED|
+                  exchange [cardinality: 1500.0, op-cost: 0.0, total-cost: 1500.0]
+                  -- HASH_PARTITION_EXCHANGE [$$43, $$56]  |PARTITIONED|
+                    assign [$$56] <- [$$43] [cardinality: 1500.0, op-cost: 0.0, total-cost: 1500.0]
+                    -- ASSIGN  |PARTITIONED|
+                      project ([$$43]) [cardinality: 1500.0, op-cost: 0.0, total-cost: 1500.0]
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$47] <- [$$l.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ASSIGN  |PARTITIONED|
-                          project ([$$44, $$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- STREAM_PROJECT  |PARTITIONED|
+                        exchange [cardinality: 1500.0, op-cost: 0.0, total-cost: 1500.0]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$43, $$o] <- tpch.Orders [cardinality: 1500.0, op-cost: 1500.0, total-cost: 1500.0]
+                          -- DATASOURCE_SCAN  |PARTITIONED|
                             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              data-scan []<-[$$44, $$45, $$l] <- tpch.LineItem [cardinality: 6005.0, op-cost: 6005.0, total-cost: 6005.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|
+                              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 [$$44, $$47]  |PARTITIONED|
+                    assign [$$47] <- [$$l.getField(2)] project: [$$44, $$47] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ASSIGN  |PARTITIONED|
+                      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 []<-[$$44, $$45, $$l] <- tpch.LineItem [cardinality: 6005.0, op-cost: 6005.0, total-cost: 6005.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_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.10.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.10.plan
index c2a7593..1bc2627 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.10.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.10.plan
@@ -1,64 +1,58 @@
-distribute result [$$51] [cardinality: 159.84, op-cost: 0.0, total-cost: 1730.95]
+distribute result [$$51] [cardinality: 166.67, op-cost: 0.0, total-cost: 1801.84]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 159.84, op-cost: 0.0, total-cost: 1730.95]
+  exchange [cardinality: 166.67, op-cost: 0.0, total-cost: 1801.84]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$51]) [cardinality: 159.84, op-cost: 0.0, total-cost: 1730.95]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$51] <- [{"n_nationkey": $$58, "s_nationkey": $$56, "c_nationkey": $$55}] [cardinality: 159.84, op-cost: 0.0, total-cost: 1730.95]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 159.84, op-cost: 0.0, total-cost: 1730.95]
-        -- SORT_MERGE_EXCHANGE [$$58(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
-          order (ASC, $$58) (ASC, $$56) (ASC, $$55) [cardinality: 159.84, op-cost: 1170.11, total-cost: 1730.95]
-          -- STABLE_SORT [$$58(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
-            exchange [cardinality: 159.84, op-cost: 0.0, total-cost: 560.84]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              join (eq($$55, $$58)) [cardinality: 159.84, op-cost: 227.92, total-cost: 560.84]
-              -- HYBRID_HASH_JOIN [$$55][$$58]  |PARTITIONED|
-                exchange [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
-                -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                  project ([$$55]) [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
+    assign [$$51] <- [{"n_nationkey": $$58, "s_nationkey": $$56, "c_nationkey": $$55}] project: [$$51] [cardinality: 166.67, op-cost: 0.0, total-cost: 1801.84]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 166.67, op-cost: 0.0, total-cost: 1801.84]
+      -- SORT_MERGE_EXCHANGE [$$58(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
+        order (ASC, $$58) (ASC, $$56) (ASC, $$55) [cardinality: 166.67, op-cost: 1230.17, total-cost: 1801.84]
+        -- STABLE_SORT [$$58(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
+          exchange [cardinality: 166.67, op-cost: 0.0, total-cost: 571.67]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            join (eq($$55, $$58)) [cardinality: 166.67, op-cost: 233.33, total-cost: 571.67]
+            -- HYBRID_HASH_JOIN [$$55][$$58]  |PARTITIONED|
+              exchange [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
+              -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                assign [$$55] <- [$$c.getField(3)] project: [$$55] [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$c]) [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$55] <- [$$c.getField(3)] [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
+                    exchange [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$60, $$c] <- tpch.Customer [cardinality: 150.0, op-cost: 150.0, total-cost: 150.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: 27.78, op-cost: 83.33, total-cost: 188.33]
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                join (eq($$56, $$58)) [cardinality: 27.78, op-cost: 35.0, total-cost: 105.0]
+                -- HYBRID_HASH_JOIN [$$58][$$56]  |PARTITIONED|
+                  exchange [cardinality: 25.0, op-cost: 25.0, total-cost: 50.0]
+                  -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
+                    project ([$$58]) [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      exchange [cardinality: 25.0, op-cost: 25.0, total-cost: 50.0]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        data-scan []<-[$$58, $$n] <- tpch.Nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.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: 10.0, op-cost: 10.0, total-cost: 20.0]
+                  -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
+                    assign [$$56] <- [$$s.getField(3)] project: [$$56] [cardinality: 10.0, op-cost: 0.0, total-cost: 10.0]
                     -- ASSIGN  |PARTITIONED|
-                      project ([$$c]) [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
+                      project ([$$s]) [cardinality: 10.0, op-cost: 0.0, total-cost: 10.0]
                       -- STREAM_PROJECT  |PARTITIONED|
-                        exchange [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
+                        exchange [cardinality: 10.0, op-cost: 10.0, total-cost: 20.0]
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$60, $$c] <- tpch.Customer [cardinality: 150.0, op-cost: 150.0, total-cost: 150.0]
+                          data-scan []<-[$$59, $$s] <- tpch.Supplier [cardinality: 10.0, op-cost: 10.0, total-cost: 10.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: 25.97, op-cost: 77.92, total-cost: 182.92]
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                  join (eq($$56, $$58)) [cardinality: 25.97, op-cost: 35.0, total-cost: 105.0]
-                  -- HYBRID_HASH_JOIN [$$58][$$56]  |PARTITIONED|
-                    exchange [cardinality: 25.0, op-cost: 25.0, total-cost: 50.0]
-                    -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-                      project ([$$58]) [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        exchange [cardinality: 25.0, op-cost: 25.0, total-cost: 50.0]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$58, $$n] <- tpch.Nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.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: 10.0, op-cost: 10.0, total-cost: 20.0]
-                    -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
-                      project ([$$56]) [cardinality: 10.0, op-cost: 0.0, total-cost: 10.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$56] <- [$$s.getField(3)] [cardinality: 10.0, op-cost: 0.0, total-cost: 10.0]
-                        -- ASSIGN  |PARTITIONED|
-                          project ([$$s]) [cardinality: 10.0, op-cost: 0.0, total-cost: 10.0]
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            exchange [cardinality: 10.0, op-cost: 10.0, total-cost: 20.0]
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              data-scan []<-[$$59, $$s] <- tpch.Supplier [cardinality: 10.0, op-cost: 10.0, total-cost: 10.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_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.12.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.12.plan
index 2e075a1..d1d77bd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.12.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.12.plan
@@ -1,64 +1,58 @@
-distribute result [$$51] [cardinality: 159.84, op-cost: 0.0, total-cost: 1777.06]
+distribute result [$$51] [cardinality: 166.67, op-cost: 0.0, total-cost: 1840.73]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 159.84, op-cost: 0.0, total-cost: 1777.06]
+  exchange [cardinality: 166.67, op-cost: 0.0, total-cost: 1840.73]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$51]) [cardinality: 159.84, op-cost: 0.0, total-cost: 1777.06]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$51] <- [{"n_nationkey": $$58, "s_nationkey": $$56, "c_nationkey": $$55}] [cardinality: 159.84, op-cost: 0.0, total-cost: 1777.06]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 159.84, op-cost: 0.0, total-cost: 1777.06]
-        -- SORT_MERGE_EXCHANGE [$$58(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
-          order (ASC, $$58) (ASC, $$56) (ASC, $$55) [cardinality: 159.84, op-cost: 1170.11, total-cost: 1777.06]
-          -- STABLE_SORT [$$58(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
-            exchange [cardinality: 159.84, op-cost: 0.0, total-cost: 606.95]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              join (eq($$55, $$58)) [cardinality: 159.84, op-cost: 175.97, total-cost: 606.95]
-              -- HYBRID_HASH_JOIN [$$58][$$55]  |PARTITIONED|
-                exchange [cardinality: 25.97, op-cost: 25.97, total-cost: 130.97]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  join (eq($$56, $$58)) [cardinality: 25.97, op-cost: 35.0, total-cost: 105.0]
-                  -- HYBRID_HASH_JOIN [$$58][$$56]  |PARTITIONED|
-                    exchange [cardinality: 25.0, op-cost: 25.0, total-cost: 50.0]
-                    -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-                      project ([$$58]) [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        exchange [cardinality: 25.0, op-cost: 25.0, total-cost: 50.0]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$58, $$n] <- tpch.Nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.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: 10.0, op-cost: 10.0, total-cost: 20.0]
-                    -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
-                      project ([$$56]) [cardinality: 10.0, op-cost: 0.0, total-cost: 10.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$56] <- [$$s.getField(3)] [cardinality: 10.0, op-cost: 0.0, total-cost: 10.0]
-                        -- ASSIGN  |PARTITIONED|
-                          project ([$$s]) [cardinality: 10.0, op-cost: 0.0, total-cost: 10.0]
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            exchange [cardinality: 10.0, op-cost: 10.0, total-cost: 20.0]
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              data-scan []<-[$$59, $$s] <- tpch.Supplier [cardinality: 10.0, op-cost: 10.0, total-cost: 10.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: 150.0, op-cost: 150.0, total-cost: 300.0]
-                -- HASH_PARTITION_EXCHANGE [$$55]  |PARTITIONED|
-                  project ([$$55]) [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$55] <- [$$c.getField(3)] [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
+    assign [$$51] <- [{"n_nationkey": $$58, "s_nationkey": $$56, "c_nationkey": $$55}] project: [$$51] [cardinality: 166.67, op-cost: 0.0, total-cost: 1840.73]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 166.67, op-cost: 0.0, total-cost: 1840.73]
+      -- SORT_MERGE_EXCHANGE [$$58(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
+        order (ASC, $$58) (ASC, $$56) (ASC, $$55) [cardinality: 166.67, op-cost: 1230.17, total-cost: 1840.73]
+        -- STABLE_SORT [$$58(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
+          exchange [cardinality: 166.67, op-cost: 0.0, total-cost: 610.56]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            join (eq($$55, $$58)) [cardinality: 166.67, op-cost: 177.78, total-cost: 610.56]
+            -- HYBRID_HASH_JOIN [$$58][$$55]  |PARTITIONED|
+              exchange [cardinality: 27.78, op-cost: 27.78, total-cost: 132.78]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                join (eq($$56, $$58)) [cardinality: 27.78, op-cost: 35.0, total-cost: 105.0]
+                -- HYBRID_HASH_JOIN [$$58][$$56]  |PARTITIONED|
+                  exchange [cardinality: 25.0, op-cost: 25.0, total-cost: 50.0]
+                  -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
+                    project ([$$58]) [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      exchange [cardinality: 25.0, op-cost: 25.0, total-cost: 50.0]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        data-scan []<-[$$58, $$n] <- tpch.Nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.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: 10.0, op-cost: 10.0, total-cost: 20.0]
+                  -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
+                    assign [$$56] <- [$$s.getField(3)] project: [$$56] [cardinality: 10.0, op-cost: 0.0, total-cost: 10.0]
                     -- ASSIGN  |PARTITIONED|
-                      project ([$$c]) [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
+                      project ([$$s]) [cardinality: 10.0, op-cost: 0.0, total-cost: 10.0]
                       -- STREAM_PROJECT  |PARTITIONED|
-                        exchange [cardinality: 150.0, op-cost: 150.0, total-cost: 300.0]
+                        exchange [cardinality: 10.0, op-cost: 10.0, total-cost: 20.0]
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$60, $$c] <- tpch.Customer [cardinality: 150.0, op-cost: 150.0, total-cost: 150.0]
+                          data-scan []<-[$$59, $$s] <- tpch.Supplier [cardinality: 10.0, op-cost: 10.0, total-cost: 10.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: 150.0, op-cost: 150.0, total-cost: 300.0]
+              -- HASH_PARTITION_EXCHANGE [$$55]  |PARTITIONED|
+                assign [$$55] <- [$$c.getField(3)] project: [$$55] [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$c]) [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    exchange [cardinality: 150.0, op-cost: 150.0, total-cost: 300.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$60, $$c] <- tpch.Customer [cardinality: 150.0, op-cost: 150.0, total-cost: 150.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_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.14.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.14.plan
index 9ca57cc..aef155b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.14.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.14.plan
@@ -1,66 +1,60 @@
-distribute result [$$51] [cardinality: 159.84, op-cost: 0.0, total-cost: 1788.03]
+distribute result [$$51] [cardinality: 166.67, op-cost: 0.0, total-cost: 1853.5]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 159.84, op-cost: 0.0, total-cost: 1788.03]
+  exchange [cardinality: 166.67, op-cost: 0.0, total-cost: 1853.5]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$51]) [cardinality: 159.84, op-cost: 0.0, total-cost: 1788.03]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$51] <- [{"n_nationkey": $$59, "s_nationkey": $$56, "c_nationkey": $$55}] [cardinality: 159.84, op-cost: 0.0, total-cost: 1788.03]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 159.84, op-cost: 0.0, total-cost: 1788.03]
-        -- SORT_MERGE_EXCHANGE [$$59(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
-          order (ASC, $$59) (ASC, $$56) (ASC, $$55) [cardinality: 159.84, op-cost: 1170.11, total-cost: 1788.03]
-          -- STABLE_SORT [$$59(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
-            exchange [cardinality: 159.84, op-cost: 0.0, total-cost: 617.92]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$59, $$56, $$55]) [cardinality: 159.84, op-cost: 0.0, total-cost: 617.92]
-              -- STREAM_PROJECT  |PARTITIONED|
-                exchange [cardinality: 159.84, op-cost: 0.0, total-cost: 617.92]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  join (and(eq($$55, $$59), eq($$56, $$66))) [cardinality: 159.84, op-cost: 175.97, total-cost: 617.92]
-                  -- HYBRID_HASH_JOIN [$$55, $$66][$$59, $$56]  |PARTITIONED|
-                    exchange [cardinality: 150.0, op-cost: 150.0, total-cost: 300.0]
-                    -- HASH_PARTITION_EXCHANGE [$$55, $$66]  |PARTITIONED|
-                      assign [$$66] <- [$$55] [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
+    assign [$$51] <- [{"n_nationkey": $$59, "s_nationkey": $$56, "c_nationkey": $$55}] project: [$$51] [cardinality: 166.67, op-cost: 0.0, total-cost: 1853.5]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 166.67, op-cost: 0.0, total-cost: 1853.5]
+      -- SORT_MERGE_EXCHANGE [$$59(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
+        order (ASC, $$59) (ASC, $$56) (ASC, $$55) [cardinality: 166.67, op-cost: 1230.17, total-cost: 1853.5]
+        -- STABLE_SORT [$$59(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
+          exchange [cardinality: 166.67, op-cost: 0.0, total-cost: 623.33]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            project ([$$59, $$56, $$55]) [cardinality: 166.67, op-cost: 0.0, total-cost: 623.33]
+            -- STREAM_PROJECT  |PARTITIONED|
+              exchange [cardinality: 166.67, op-cost: 0.0, total-cost: 623.33]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                join (and(eq($$55, $$59), eq($$56, $$66))) [cardinality: 166.67, op-cost: 177.78, total-cost: 623.33]
+                -- HYBRID_HASH_JOIN [$$55, $$66][$$59, $$56]  |PARTITIONED|
+                  exchange [cardinality: 150.0, op-cost: 150.0, total-cost: 300.0]
+                  -- HASH_PARTITION_EXCHANGE [$$55, $$66]  |PARTITIONED|
+                    assign [$$66] <- [$$55] [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
+                    -- ASSIGN  |PARTITIONED|
+                      assign [$$55] <- [$$c.getField(3)] project: [$$55] [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
                       -- ASSIGN  |PARTITIONED|
-                        project ([$$55]) [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
+                        project ([$$c]) [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
                         -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$55] <- [$$c.getField(3)] [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
-                          -- ASSIGN  |PARTITIONED|
-                            project ([$$c]) [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              exchange [cardinality: 150.0, op-cost: 150.0, total-cost: 300.0]
+                          exchange [cardinality: 150.0, op-cost: 150.0, total-cost: 300.0]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            data-scan []<-[$$60, $$c] <- tpch.Customer [cardinality: 150.0, op-cost: 150.0, total-cost: 150.0]
+                            -- DATASOURCE_SCAN  |PARTITIONED|
+                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                data-scan []<-[$$60, $$c] <- tpch.Customer [cardinality: 150.0, op-cost: 150.0, total-cost: 150.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: 25.97, op-cost: 25.97, total-cost: 141.94]
-                    -- HASH_PARTITION_EXCHANGE [$$59, $$56]  |PARTITIONED|
-                      project ([$$56, $$59]) [cardinality: 25.97, op-cost: 0.0, total-cost: 115.97]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        exchange [cardinality: 25.97, op-cost: 25.97, total-cost: 141.94]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          unnest-map [$$59, $$n] <- index-search("Nation", 0, "Default", "tpch", "Nation", true, true, 1, $$56, 1, $$56, true, true, true) [cardinality: 25.97, op-cost: 65.97, total-cost: 115.97]
-                          -- BTREE_SEARCH  |PARTITIONED|
-                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              order (ASC, $$56) [cardinality: 25.97, op-cost: 65.97, total-cost: 115.97]
-                              -- STABLE_SORT [$$56(ASC)]  |PARTITIONED|
-                                exchange [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]
+                                empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  exchange [cardinality: 27.78, op-cost: 27.78, total-cost: 145.56]
+                  -- HASH_PARTITION_EXCHANGE [$$59, $$56]  |PARTITIONED|
+                    project ([$$56, $$59]) [cardinality: 27.78, op-cost: 0.0, total-cost: 117.78]
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      exchange [cardinality: 27.78, op-cost: 27.78, total-cost: 145.56]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        unnest-map [$$59, $$n] <- index-search("Nation", 0, "Default", "tpch", "Nation", true, true, 1, $$56, 1, $$56, true, true, true) [cardinality: 27.78, op-cost: 67.78, total-cost: 117.78]
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            order (ASC, $$56) [cardinality: 27.78, op-cost: 67.78, total-cost: 117.78]
+                            -- STABLE_SORT [$$56(ASC)]  |PARTITIONED|
+                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
+                                assign [$$56] <- [$$s.getField(3)] project: [$$56] [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|
-                                    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 []<-[$$58, $$s] <- tpch.Supplier [cardinality: 10.0, op-cost: 10.0, total-cost: 10.0]
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
                                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          data-scan []<-[$$58, $$s] <- tpch.Supplier [cardinality: 10.0, op-cost: 10.0, total-cost: 10.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|
+                                          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_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.16.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.16.plan
index dd41dfc..8a4aa88 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.16.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.16.plan
@@ -1,66 +1,60 @@
-distribute result [$$51] [cardinality: 159.84, op-cost: 0.0, total-cost: 1731.93]
+distribute result [$$51] [cardinality: 166.67, op-cost: 0.0, total-cost: 1804.61]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 159.84, op-cost: 0.0, total-cost: 1731.93]
+  exchange [cardinality: 166.67, op-cost: 0.0, total-cost: 1804.61]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$51]) [cardinality: 159.84, op-cost: 0.0, total-cost: 1731.93]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$51] <- [{"n_nationkey": $$59, "s_nationkey": $$56, "c_nationkey": $$55}] [cardinality: 159.84, op-cost: 0.0, total-cost: 1731.93]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 159.84, op-cost: 0.0, total-cost: 1731.93]
-        -- SORT_MERGE_EXCHANGE [$$59(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
-          order (ASC, $$59) (ASC, $$56) (ASC, $$55) [cardinality: 159.84, op-cost: 1170.11, total-cost: 1731.93]
-          -- STABLE_SORT [$$59(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
-            exchange [cardinality: 159.84, op-cost: 0.0, total-cost: 561.82]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$59, $$56, $$55]) [cardinality: 159.84, op-cost: 0.0, total-cost: 561.82]
-              -- STREAM_PROJECT  |PARTITIONED|
-                exchange [cardinality: 159.84, op-cost: 0.0, total-cost: 561.82]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  join (and(eq($$55, $$59), eq($$56, $$66))) [cardinality: 159.84, op-cost: 227.92, total-cost: 561.82]
-                  -- HYBRID_HASH_JOIN [$$55, $$66][$$59, $$56]  |PARTITIONED|
-                    exchange [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
-                    -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                      assign [$$66] <- [$$55] [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
+    assign [$$51] <- [{"n_nationkey": $$59, "s_nationkey": $$56, "c_nationkey": $$55}] project: [$$51] [cardinality: 166.67, op-cost: 0.0, total-cost: 1804.61]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 166.67, op-cost: 0.0, total-cost: 1804.61]
+      -- SORT_MERGE_EXCHANGE [$$59(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
+        order (ASC, $$59) (ASC, $$56) (ASC, $$55) [cardinality: 166.67, op-cost: 1230.17, total-cost: 1804.61]
+        -- STABLE_SORT [$$59(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
+          exchange [cardinality: 166.67, op-cost: 0.0, total-cost: 574.44]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            project ([$$59, $$56, $$55]) [cardinality: 166.67, op-cost: 0.0, total-cost: 574.44]
+            -- STREAM_PROJECT  |PARTITIONED|
+              exchange [cardinality: 166.67, op-cost: 0.0, total-cost: 574.44]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                join (and(eq($$55, $$59), eq($$56, $$66))) [cardinality: 166.67, op-cost: 233.33, total-cost: 574.44]
+                -- HYBRID_HASH_JOIN [$$55, $$66][$$59, $$56]  |PARTITIONED|
+                  exchange [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
+                  -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                    assign [$$66] <- [$$55] [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
+                    -- ASSIGN  |PARTITIONED|
+                      assign [$$55] <- [$$c.getField(3)] project: [$$55] [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
                       -- ASSIGN  |PARTITIONED|
-                        project ([$$55]) [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
+                        project ([$$c]) [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
                         -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$55] <- [$$c.getField(3)] [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
-                          -- ASSIGN  |PARTITIONED|
-                            project ([$$c]) [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              exchange [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
+                          exchange [cardinality: 150.0, op-cost: 0.0, total-cost: 150.0]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            data-scan []<-[$$60, $$c] <- tpch.Customer [cardinality: 150.0, op-cost: 150.0, total-cost: 150.0]
+                            -- DATASOURCE_SCAN  |PARTITIONED|
+                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                data-scan []<-[$$60, $$c] <- tpch.Customer [cardinality: 150.0, op-cost: 150.0, total-cost: 150.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: 25.97, op-cost: 77.92, total-cost: 183.89]
-                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                      project ([$$56, $$59]) [cardinality: 25.97, op-cost: 0.0, total-cost: 105.97]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        exchange [cardinality: 25.97, op-cost: 77.92, total-cost: 183.89]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          unnest-map [$$59, $$n] <- index-search("Nation", 0, "Default", "tpch", "Nation", true, true, 1, $$56, 1, $$56, true, true, true) [cardinality: 25.97, op-cost: 65.97, total-cost: 105.97]
-                          -- BTREE_SEARCH  |PARTITIONED|
-                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              order (ASC, $$56) [cardinality: 25.97, op-cost: 65.97, total-cost: 105.97]
-                              -- STABLE_SORT [$$56(ASC)]  |PARTITIONED|
-                                exchange [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]
+                                empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  exchange [cardinality: 27.78, op-cost: 83.33, total-cost: 191.11]
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    project ([$$56, $$59]) [cardinality: 27.78, op-cost: 0.0, total-cost: 107.78]
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      exchange [cardinality: 27.78, op-cost: 83.33, total-cost: 191.11]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        unnest-map [$$59, $$n] <- index-search("Nation", 0, "Default", "tpch", "Nation", true, true, 1, $$56, 1, $$56, true, true, true) [cardinality: 27.78, op-cost: 67.78, total-cost: 107.78]
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            order (ASC, $$56) [cardinality: 27.78, op-cost: 67.78, total-cost: 107.78]
+                            -- STABLE_SORT [$$56(ASC)]  |PARTITIONED|
+                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
+                                assign [$$56] <- [$$s.getField(3)] project: [$$56] [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|
-                                    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 []<-[$$58, $$s] <- tpch.Supplier [cardinality: 10.0, op-cost: 10.0, total-cost: 10.0]
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
                                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          data-scan []<-[$$58, $$s] <- tpch.Supplier [cardinality: 10.0, op-cost: 10.0, total-cost: 10.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|
+                                          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_cbo/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.007.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.007.plan
index 17d8f1c..160636d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.007.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.007.plan
@@ -1,62 +1,54 @@
-distribute result [$$52] [cardinality: 4.0, op-cost: 0.0, total-cost: 30.0]
+distribute result [$$52] [cardinality: 6.0, op-cost: 0.0, total-cost: 39.51]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 4.0, op-cost: 0.0, total-cost: 30.0]
+  exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 39.51]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$52]) [cardinality: 4.0, op-cost: 0.0, total-cost: 30.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$52] <- [{"t1_id": $$53, "t2_id": $$54}] [cardinality: 4.0, op-cost: 0.0, total-cost: 30.0]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 4.0, op-cost: 0.0, total-cost: 30.0]
-        -- SORT_MERGE_EXCHANGE [$$53(ASC), $$54(ASC) ]  |PARTITIONED|
-          order (ASC, $$53) (ASC, $$54) [cardinality: 4.0, op-cost: 8.0, total-cost: 30.0]
-          -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
-            exchange [cardinality: 4.0, op-cost: 0.0, total-cost: 22.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$53, $$54]) [cardinality: 4.0, op-cost: 0.0, total-cost: 22.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select ($$62) [cardinality: 4.0, op-cost: 12.0, total-cost: 22.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  window-aggregate [$$62] <- [win-mark-first-missing-impl($$54)] partition [$$53] order (DESC, $$54) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                  -- WINDOW_STREAM  |PARTITIONED|
+    assign [$$52] <- [{"t1_id": $$53, "t2_id": $$54}] project: [$$52] [cardinality: 6.0, op-cost: 0.0, total-cost: 39.51]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 39.51]
+      -- SORT_MERGE_EXCHANGE [$$53(ASC), $$54(ASC) ]  |PARTITIONED|
+        order (ASC, $$53) (ASC, $$54) [cardinality: 6.0, op-cost: 15.51, total-cost: 39.51]
+        -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
+          exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 24.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select ($$62) project: [$$53, $$54] [cardinality: 6.0, op-cost: 14.0, total-cost: 24.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              window-aggregate [$$62] <- [win-mark-first-missing-impl($$54)] partition [$$53] order (DESC, $$54) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+              -- WINDOW_STREAM  |PARTITIONED|
+                exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$53) (DESC, $$54) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                  -- STABLE_SORT [$$53(ASC), $$54(DESC)]  |PARTITIONED|
                     exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      order (ASC, $$53) (DESC, $$54) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                      -- STABLE_SORT [$$53(ASC), $$54(DESC)]  |PARTITIONED|
-                        exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                        -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
-                          project ([$$53, $$54]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            select (eq($$55, $$t2.getField("c_int32"))) retain-untrue ($$54 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_SELECT  |PARTITIONED|
-                              project ([$$53, $$55, $$54, $$t2]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  left-outer-unnest-map [$$54, $$t2] <- index-search("ds_inner_untyped", 0, "Default", "test", "ds_inner_untyped", true, false, 1, $$61, 1, $$61, true, true, true) [cardinality: 3.0, op-cost: 3.0, total-cost: 3.0]
-                                  -- BTREE_SEARCH  |PARTITIONED|
-                                    exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      order (ASC, $$61) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                      -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
-                                        exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          project ([$$53, $$55, $$61]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              left-outer-unnest-map [$$60, $$61] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_untyped", true, true, 1, $$55, 1, $$55, true, true, true) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- BTREE_SEARCH  |PARTITIONED|
-                                                exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                  project ([$$53, $$55]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    assign [$$55] <- [int32($$t1.getField("c_int32"))] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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|
+                    -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                      select (eq($$55, $$t2.getField("c_int32"))) retain-untrue ($$54 <- missing) project: [$$53, $$54] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- STREAM_SELECT  |PARTITIONED|
+                        project ([$$53, $$55, $$54, $$t2]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            left-outer-unnest-map [$$54, $$t2] <- index-search("ds_inner_untyped", 0, "Default", "test", "ds_inner_untyped", true, false, 1, $$61, 1, $$61, true, true, true) [cardinality: 3.0, op-cost: 3.0, total-cost: 3.0]
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                order (ASC, $$61) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
+                                  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    project ([$$53, $$55, $$61]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        left-outer-unnest-map [$$60, $$61] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_untyped", true, true, 1, $$55, 1, $$55, true, true, true) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            assign [$$55] <- [int32($$t1.getField("c_int32"))] project: [$$53, $$55] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                            -- ASSIGN  |PARTITIONED|
+                                              exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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_cbo/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.008.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.008.plan
index 66414a5..5efb1b8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.008.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.008.plan
@@ -1,76 +1,40 @@
-distribute result [$$52] [cardinality: 4.0, op-cost: 0.0, total-cost: 30.0]
+distribute result [$$52] [cardinality: 6.0, op-cost: 0.0, total-cost: 39.51]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 4.0, op-cost: 0.0, total-cost: 30.0]
+  exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 39.51]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$52]) [cardinality: 4.0, op-cost: 0.0, total-cost: 30.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$52] <- [{"t1_id": $$73, "t2_id": $$54}] [cardinality: 4.0, op-cost: 0.0, total-cost: 30.0]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 4.0, op-cost: 0.0, total-cost: 30.0]
-        -- SORT_MERGE_EXCHANGE [$$73(ASC), $$54(ASC) ]  |PARTITIONED|
-          order (ASC, $$73) (ASC, $$54) [cardinality: 4.0, op-cost: 8.0, total-cost: 30.0]
-          -- STABLE_SORT [$$73(ASC), $$54(ASC)]  |PARTITIONED|
-            exchange [cardinality: 4.0, op-cost: 0.0, total-cost: 22.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$73, $$54]) [cardinality: 4.0, op-cost: 0.0, total-cost: 22.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select ($$74) [cardinality: 4.0, op-cost: 12.0, total-cost: 22.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  window-aggregate [$$74] <- [win-mark-first-missing-impl($$54)] partition [$$73] order (DESC, $$54) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                  -- WINDOW_STREAM  |PARTITIONED|
+    assign [$$52] <- [{"t1_id": $$73, "t2_id": $$54}] project: [$$52] [cardinality: 6.0, op-cost: 0.0, total-cost: 39.51]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 39.51]
+      -- SORT_MERGE_EXCHANGE [$$73(ASC), $$54(ASC) ]  |PARTITIONED|
+        order (ASC, $$73) (ASC, $$54) [cardinality: 6.0, op-cost: 15.51, total-cost: 39.51]
+        -- STABLE_SORT [$$73(ASC), $$54(ASC)]  |PARTITIONED|
+          exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 24.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select ($$74) project: [$$73, $$54] [cardinality: 6.0, op-cost: 14.0, total-cost: 24.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              window-aggregate [$$74] <- [win-mark-first-missing-impl($$54)] partition [$$73] order (DESC, $$54) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+              -- WINDOW_STREAM  |PARTITIONED|
+                exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$73) (DESC, $$54) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                  -- STABLE_SORT [$$73(ASC), $$54(DESC)]  |PARTITIONED|
                     exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      order (ASC, $$73) (DESC, $$54) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                      -- STABLE_SORT [$$73(ASC), $$54(DESC)]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$73]  |PARTITIONED|
+                      union ($$70, $$61, $$54) ($$53, $$53, $$73) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                      -- UNION_ALL  |PARTITIONED|
                         exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                        -- HASH_PARTITION_EXCHANGE [$$73]  |PARTITIONED|
-                          union ($$70, $$61, $$54) ($$53, $$53, $$73) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                          -- UNION_ALL  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          select (eq($$55, $$71.getField(1))) retain-untrue ($$70 <- missing) project: [$$70, $$53] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- STREAM_SELECT  |PARTITIONED|
                             exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              project ([$$70, $$53]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                select (eq($$55, $$71.getField(1))) retain-untrue ($$70 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    left-outer-unnest-map [$$70, $$71] <- index-search("ds_inner_typed", 0, "Default", "test", "ds_inner_typed", true, false, 1, $$61, 1, $$61, true, true, true) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- BTREE_SEARCH  |PARTITIONED|
-                                      exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        project ([$$53, $$55, $$61]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            split ($$62) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                            -- SPLIT  |PARTITIONED|
-                                              exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                left-outer-unnest-map [$$60, $$61, $$62] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_typed", true, true, 1, $$55, 1, $$55, true, true, true) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- BTREE_SEARCH  |PARTITIONED|
-                                                  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                    project ([$$53, $$55]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      assign [$$55] <- [int32($$t1.getField("c_int32"))] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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  |PARTITIONED|
-                              project ([$$61, $$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                select (eq($$55, $$60)) retain-untrue ($$61 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  project ([$$53, $$55, $$60, $$61]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              left-outer-unnest-map [$$70, $$71] <- index-search("ds_inner_typed", 0, "Default", "test", "ds_inner_typed", true, false, 1, $$61, 1, $$61, true, true, true) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- BTREE_SEARCH  |PARTITIONED|
+                                exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  project ([$$53, $$55, $$61]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                                   -- STREAM_PROJECT  |PARTITIONED|
-                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       split ($$62) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                                       -- SPLIT  |PARTITIONED|
@@ -80,15 +44,39 @@
                                           -- BTREE_SEARCH  |PARTITIONED|
                                             exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                                             -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                              project ([$$53, $$55]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                assign [$$55] <- [int32($$t1.getField("c_int32"))] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                                -- ASSIGN  |PARTITIONED|
-                                                  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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|
+                                              assign [$$55] <- [int32($$t1.getField("c_int32"))] project: [$$53, $$55] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                              -- ASSIGN  |PARTITIONED|
+                                                exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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  |PARTITIONED|
+                          select (eq($$55, $$60)) retain-untrue ($$61 <- missing) project: [$$61, $$53] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- STREAM_SELECT  |PARTITIONED|
+                            project ([$$53, $$55, $$60, $$61]) [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|
+                                split ($$62) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                -- SPLIT  |PARTITIONED|
+                                  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    left-outer-unnest-map [$$60, $$61, $$62] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_typed", true, true, 1, $$55, 1, $$55, true, true, true) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- BTREE_SEARCH  |PARTITIONED|
+                                      exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        assign [$$55] <- [int32($$t1.getField("c_int32"))] project: [$$53, $$55] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                        -- ASSIGN  |PARTITIONED|
+                                          exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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_cbo/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.009.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.009.plan
index 814461c..76995db 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.009.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.009.plan
@@ -1,62 +1,54 @@
-distribute result [$$52] [cardinality: 4.0, op-cost: 0.0, total-cost: 30.0]
+distribute result [$$52] [cardinality: 6.0, op-cost: 0.0, total-cost: 39.51]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 4.0, op-cost: 0.0, total-cost: 30.0]
+  exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 39.51]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$52]) [cardinality: 4.0, op-cost: 0.0, total-cost: 30.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$52] <- [{"t1_id": $$53, "t2_id": $$54}] [cardinality: 4.0, op-cost: 0.0, total-cost: 30.0]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 4.0, op-cost: 0.0, total-cost: 30.0]
-        -- SORT_MERGE_EXCHANGE [$$53(ASC), $$54(ASC) ]  |PARTITIONED|
-          order (ASC, $$53) (ASC, $$54) [cardinality: 4.0, op-cost: 8.0, total-cost: 30.0]
-          -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
-            exchange [cardinality: 4.0, op-cost: 0.0, total-cost: 22.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$53, $$54]) [cardinality: 4.0, op-cost: 0.0, total-cost: 22.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select ($$62) [cardinality: 4.0, op-cost: 12.0, total-cost: 22.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  window-aggregate [$$62] <- [win-mark-first-missing-impl($$54)] partition [$$53] order (DESC, $$54) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                  -- WINDOW_STREAM  |PARTITIONED|
+    assign [$$52] <- [{"t1_id": $$53, "t2_id": $$54}] project: [$$52] [cardinality: 6.0, op-cost: 0.0, total-cost: 39.51]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 39.51]
+      -- SORT_MERGE_EXCHANGE [$$53(ASC), $$54(ASC) ]  |PARTITIONED|
+        order (ASC, $$53) (ASC, $$54) [cardinality: 6.0, op-cost: 15.51, total-cost: 39.51]
+        -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
+          exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 24.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select ($$62) project: [$$53, $$54] [cardinality: 6.0, op-cost: 14.0, total-cost: 24.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              window-aggregate [$$62] <- [win-mark-first-missing-impl($$54)] partition [$$53] order (DESC, $$54) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+              -- WINDOW_STREAM  |PARTITIONED|
+                exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$53) (DESC, $$54) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                  -- STABLE_SORT [$$53(ASC), $$54(DESC)]  |PARTITIONED|
                     exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      order (ASC, $$53) (DESC, $$54) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                      -- STABLE_SORT [$$53(ASC), $$54(DESC)]  |PARTITIONED|
-                        exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                        -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
-                          project ([$$53, $$54]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            select (eq($$55, $$t2.getField("c_int32"))) retain-untrue ($$54 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_SELECT  |PARTITIONED|
-                              project ([$$53, $$55, $$54, $$t2]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  left-outer-unnest-map [$$54, $$t2] <- index-search("ds_inner_untyped", 0, "Default", "test", "ds_inner_untyped", true, false, 1, $$61, 1, $$61, true, true, true) [cardinality: 3.0, op-cost: 3.0, total-cost: 3.0]
-                                  -- BTREE_SEARCH  |PARTITIONED|
-                                    exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      order (ASC, $$61) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                      -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
-                                        exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          project ([$$53, $$55, $$61]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              left-outer-unnest-map [$$60, $$61] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_untyped", true, true, 1, $$55, 1, $$55, true, true, true) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- BTREE_SEARCH  |PARTITIONED|
-                                                exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                  project ([$$53, $$55]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    assign [$$55] <- [int32($$t1.getField(1))] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        data-scan []<-[$$53, $$t1] <- test.ds_outer_typed [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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|
+                    -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                      select (eq($$55, $$t2.getField("c_int32"))) retain-untrue ($$54 <- missing) project: [$$53, $$54] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- STREAM_SELECT  |PARTITIONED|
+                        project ([$$53, $$55, $$54, $$t2]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            left-outer-unnest-map [$$54, $$t2] <- index-search("ds_inner_untyped", 0, "Default", "test", "ds_inner_untyped", true, false, 1, $$61, 1, $$61, true, true, true) [cardinality: 3.0, op-cost: 3.0, total-cost: 3.0]
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                order (ASC, $$61) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
+                                  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    project ([$$53, $$55, $$61]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        left-outer-unnest-map [$$60, $$61] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_untyped", true, true, 1, $$55, 1, $$55, true, true, true) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            assign [$$55] <- [int32($$t1.getField(1))] project: [$$53, $$55] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                            -- ASSIGN  |PARTITIONED|
+                                              exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                data-scan []<-[$$53, $$t1] <- test.ds_outer_typed [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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_cbo/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.010.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.010.plan
index a6e0902..567ccef 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.010.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.010.plan
@@ -1,76 +1,40 @@
-distribute result [$$52] [cardinality: 4.0, op-cost: 0.0, total-cost: 30.0]
+distribute result [$$52] [cardinality: 6.0, op-cost: 0.0, total-cost: 39.51]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 4.0, op-cost: 0.0, total-cost: 30.0]
+  exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 39.51]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$52]) [cardinality: 4.0, op-cost: 0.0, total-cost: 30.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$52] <- [{"t1_id": $$73, "t2_id": $$54}] [cardinality: 4.0, op-cost: 0.0, total-cost: 30.0]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 4.0, op-cost: 0.0, total-cost: 30.0]
-        -- SORT_MERGE_EXCHANGE [$$73(ASC), $$54(ASC) ]  |PARTITIONED|
-          order (ASC, $$73) (ASC, $$54) [cardinality: 4.0, op-cost: 8.0, total-cost: 30.0]
-          -- STABLE_SORT [$$73(ASC), $$54(ASC)]  |PARTITIONED|
-            exchange [cardinality: 4.0, op-cost: 0.0, total-cost: 22.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$73, $$54]) [cardinality: 4.0, op-cost: 0.0, total-cost: 22.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select ($$74) [cardinality: 4.0, op-cost: 12.0, total-cost: 22.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  window-aggregate [$$74] <- [win-mark-first-missing-impl($$54)] partition [$$73] order (DESC, $$54) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                  -- WINDOW_STREAM  |PARTITIONED|
+    assign [$$52] <- [{"t1_id": $$73, "t2_id": $$54}] project: [$$52] [cardinality: 6.0, op-cost: 0.0, total-cost: 39.51]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 39.51]
+      -- SORT_MERGE_EXCHANGE [$$73(ASC), $$54(ASC) ]  |PARTITIONED|
+        order (ASC, $$73) (ASC, $$54) [cardinality: 6.0, op-cost: 15.51, total-cost: 39.51]
+        -- STABLE_SORT [$$73(ASC), $$54(ASC)]  |PARTITIONED|
+          exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 24.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select ($$74) project: [$$73, $$54] [cardinality: 6.0, op-cost: 14.0, total-cost: 24.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              window-aggregate [$$74] <- [win-mark-first-missing-impl($$54)] partition [$$73] order (DESC, $$54) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+              -- WINDOW_STREAM  |PARTITIONED|
+                exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$73) (DESC, $$54) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                  -- STABLE_SORT [$$73(ASC), $$54(DESC)]  |PARTITIONED|
                     exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      order (ASC, $$73) (DESC, $$54) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                      -- STABLE_SORT [$$73(ASC), $$54(DESC)]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$73]  |PARTITIONED|
+                      union ($$70, $$61, $$54) ($$53, $$53, $$73) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                      -- UNION_ALL  |PARTITIONED|
                         exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                        -- HASH_PARTITION_EXCHANGE [$$73]  |PARTITIONED|
-                          union ($$70, $$61, $$54) ($$53, $$53, $$73) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                          -- UNION_ALL  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          select (eq($$55, $$71.getField(1))) retain-untrue ($$70 <- missing) project: [$$70, $$53] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- STREAM_SELECT  |PARTITIONED|
                             exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              project ([$$70, $$53]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                select (eq($$55, $$71.getField(1))) retain-untrue ($$70 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    left-outer-unnest-map [$$70, $$71] <- index-search("ds_inner_typed", 0, "Default", "test", "ds_inner_typed", true, false, 1, $$61, 1, $$61, true, true, true) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- BTREE_SEARCH  |PARTITIONED|
-                                      exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        project ([$$53, $$55, $$61]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            split ($$62) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                            -- SPLIT  |PARTITIONED|
-                                              exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                left-outer-unnest-map [$$60, $$61, $$62] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_typed", true, true, 1, $$55, 1, $$55, true, true, true) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- BTREE_SEARCH  |PARTITIONED|
-                                                  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                    project ([$$53, $$55]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      assign [$$55] <- [int32($$t1.getField(1))] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          data-scan []<-[$$53, $$t1] <- test.ds_outer_typed [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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  |PARTITIONED|
-                              project ([$$61, $$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                select (eq($$55, $$60)) retain-untrue ($$61 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  project ([$$53, $$55, $$60, $$61]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              left-outer-unnest-map [$$70, $$71] <- index-search("ds_inner_typed", 0, "Default", "test", "ds_inner_typed", true, false, 1, $$61, 1, $$61, true, true, true) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- BTREE_SEARCH  |PARTITIONED|
+                                exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  project ([$$53, $$55, $$61]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                                   -- STREAM_PROJECT  |PARTITIONED|
-                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       split ($$62) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                                       -- SPLIT  |PARTITIONED|
@@ -80,15 +44,39 @@
                                           -- BTREE_SEARCH  |PARTITIONED|
                                             exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
                                             -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                              project ([$$53, $$55]) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                assign [$$55] <- [int32($$t1.getField(1))] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                                -- ASSIGN  |PARTITIONED|
-                                                  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    data-scan []<-[$$53, $$t1] <- test.ds_outer_typed [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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|
+                                              assign [$$55] <- [int32($$t1.getField(1))] project: [$$53, $$55] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                              -- ASSIGN  |PARTITIONED|
+                                                exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  data-scan []<-[$$53, $$t1] <- test.ds_outer_typed [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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  |PARTITIONED|
+                          select (eq($$55, $$60)) retain-untrue ($$61 <- missing) project: [$$61, $$53] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- STREAM_SELECT  |PARTITIONED|
+                            project ([$$53, $$55, $$60, $$61]) [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|
+                                split ($$62) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                -- SPLIT  |PARTITIONED|
+                                  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    left-outer-unnest-map [$$60, $$61, $$62] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_typed", true, true, 1, $$55, 1, $$55, true, true, true) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- BTREE_SEARCH  |PARTITIONED|
+                                      exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        assign [$$55] <- [int32($$t1.getField(1))] project: [$$53, $$55] [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                        -- ASSIGN  |PARTITIONED|
+                                          exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            data-scan []<-[$$53, $$t1] <- test.ds_outer_typed [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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_cbo/limit/offset_without_limit/offset_without_limit.6.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/limit/offset_without_limit/offset_without_limit.6.plan
index 57567e5..0ad749a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/limit/offset_without_limit/offset_without_limit.6.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/limit/offset_without_limit/offset_without_limit.6.plan
@@ -4,19 +4,17 @@
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit offset 98 [cardinality: 100.0, op-cost: 0.0, total-cost: 764.39]
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$16]) [cardinality: 100.0, op-cost: 0.0, total-cost: 764.39]
-      -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$16] <- [{"id": $$18, "dblpid": $$paper.getField(1)}] [cardinality: 100.0, op-cost: 0.0, total-cost: 764.39]
-        -- ASSIGN  |PARTITIONED|
-          exchange [cardinality: 100.0, op-cost: 0.0, total-cost: 764.39]
-          -- SORT_MERGE_EXCHANGE [$$18(ASC) ]  |PARTITIONED|
-            order (ASC, $$18) [cardinality: 100.0, op-cost: 664.39, total-cost: 764.39]
-            -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
-              exchange [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                data-scan []<-[$$18, $$paper] <- test.DBLP1 [cardinality: 100.0, op-cost: 100.0, total-cost: 100.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|
+      assign [$$16] <- [{"id": $$18, "dblpid": $$paper.getField(1)}] project: [$$16] [cardinality: 100.0, op-cost: 0.0, total-cost: 764.39]
+      -- ASSIGN  |PARTITIONED|
+        exchange [cardinality: 100.0, op-cost: 0.0, total-cost: 764.39]
+        -- SORT_MERGE_EXCHANGE [$$18(ASC) ]  |PARTITIONED|
+          order (ASC, $$18) [cardinality: 100.0, op-cost: 664.39, total-cost: 764.39]
+          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+            exchange [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$18, $$paper] <- test.DBLP1 [cardinality: 100.0, op-cost: 100.0, total-cost: 100.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_cbo/limit/push-limit-to-external-scan-select/push-limit-to-external-scan-select.2.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/limit/push-limit-to-external-scan-select/push-limit-to-external-scan-select.2.plan
index 11d4b1a..82aa37e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/limit/push-limit-to-external-scan-select/push-limit-to-external-scan-select.2.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/limit/push-limit-to-external-scan-select/push-limit-to-external-scan-select.2.plan
@@ -8,15 +8,13 @@
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
         limit 5 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
         -- STREAM_LIMIT  |PARTITIONED|
-          project ([$$17]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STREAM_PROJECT  |PARTITIONED|
-            assign [$$17] <- [$$t.getField(0)] [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]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                data-scan []<-[$$t] <- test.ds1 condition (gt($$t.getField(0), 2)) limit 5 [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|
+          assign [$$17] <- [$$t.getField(0)] project: [$$17] [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]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$t] <- test.ds1 condition (gt($$t.getField(0), 2)) limit 5 [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_cbo/limit/push-limit-to-external-scan/push-limit-to-external-scan.2.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/limit/push-limit-to-external-scan/push-limit-to-external-scan.2.plan
index 55b2c18..4428df0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/limit/push-limit-to-external-scan/push-limit-to-external-scan.2.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/limit/push-limit-to-external-scan/push-limit-to-external-scan.2.plan
@@ -6,17 +6,15 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        project ([$$14]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          assign [$$14] <- [$$t.getField(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|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                data-scan []<-[$$t] <- test.ds1 limit 5 [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|
+        assign [$$14] <- [$$t.getField(0)] project: [$$14] [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|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$t] <- test.ds1 limit 5 [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_cbo/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.plan
index 653da45..251dee3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.plan
@@ -6,21 +6,19 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       exchange [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        project ([$$20]) [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          assign [$$20] <- [{"shipdate": substring($$c.getField(10), 0, 4), "suppkey": gt($$21, 0)}] [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
-          -- ASSIGN  |PARTITIONED|
-            limit 5 [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
-            -- STREAM_LIMIT  |PARTITIONED|
-              assign [$$21] <- [$$c.getField(2)] [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
-              -- ASSIGN  |PARTITIONED|
-                project ([$$c]) [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
-                -- STREAM_PROJECT  |PARTITIONED|
-                  exchange [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$22, $$23, $$c] <- test.LineItem condition (lt($$c.getField(2), 150)) limit 5 [cardinality: 6010.65, op-cost: 6005.0, total-cost: 6005.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|
+        assign [$$20] <- [{"shipdate": substring($$c.getField(10), 0, 4), "suppkey": gt($$21, 0)}] project: [$$20] [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
+        -- ASSIGN  |PARTITIONED|
+          limit 5 [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
+          -- STREAM_LIMIT  |PARTITIONED|
+            assign [$$21] <- [$$c.getField(2)] [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
+            -- ASSIGN  |PARTITIONED|
+              project ([$$c]) [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
+              -- STREAM_PROJECT  |PARTITIONED|
+                exchange [cardinality: 6010.65, op-cost: 0.0, total-cost: 6005.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$22, $$23, $$c] <- test.LineItem condition (lt($$c.getField(2), 150)) limit 5 [cardinality: 6010.65, op-cost: 6005.0, total-cost: 6005.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_cbo/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.5.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.5.plan
index 035ab6e..abe93b6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.5.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.5.plan
@@ -1,64 +1,56 @@
-distribute result [$$37] [cardinality: 63.37, op-cost: 0.0, total-cost: 308.0]
+distribute result [$$37] [cardinality: 100.0, op-cost: 0.0, total-cost: 308.0]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
-  exchange [cardinality: 63.37, op-cost: 0.0, total-cost: 308.0]
+  exchange [cardinality: 100.0, op-cost: 0.0, total-cost: 308.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    limit 2 [cardinality: 63.37, op-cost: 0.0, total-cost: 308.0]
+    limit 2 [cardinality: 100.0, op-cost: 0.0, total-cost: 308.0]
     -- STREAM_LIMIT  |UNPARTITIONED|
-      exchange [cardinality: 63.37, op-cost: 0.0, total-cost: 308.0]
+      exchange [cardinality: 100.0, op-cost: 0.0, total-cost: 308.0]
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        project ([$$37]) [cardinality: 63.37, op-cost: 0.0, total-cost: 308.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          assign [$$37] <- [{"dblpid": $$38}] [cardinality: 63.37, op-cost: 0.0, total-cost: 308.0]
-          -- ASSIGN  |PARTITIONED|
-            limit 2 [cardinality: 63.37, op-cost: 0.0, total-cost: 308.0]
-            -- STREAM_LIMIT  |PARTITIONED|
-              project ([$$38]) [cardinality: 63.37, op-cost: 0.0, total-cost: 308.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                exchange [cardinality: 63.37, op-cost: 0.0, total-cost: 308.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  join (eq($$38, $$41)) [cardinality: 63.37, op-cost: 104.0, total-cost: 308.0]
-                  -- HYBRID_HASH_JOIN [$$38][$$41]  |PARTITIONED|
-                    exchange [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      project ([$$38]) [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
+        assign [$$37] <- [{"dblpid": $$38}] project: [$$37] [cardinality: 100.0, op-cost: 0.0, total-cost: 308.0]
+        -- ASSIGN  |PARTITIONED|
+          limit 2 [cardinality: 100.0, op-cost: 0.0, total-cost: 308.0]
+          -- STREAM_LIMIT  |PARTITIONED|
+            project ([$$38]) [cardinality: 100.0, op-cost: 0.0, total-cost: 308.0]
+            -- STREAM_PROJECT  |PARTITIONED|
+              exchange [cardinality: 100.0, op-cost: 0.0, total-cost: 308.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                join (eq($$38, $$41)) [cardinality: 100.0, op-cost: 104.0, total-cost: 308.0]
+                -- HYBRID_HASH_JOIN [$$38][$$41]  |PARTITIONED|
+                  exchange [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    assign [$$38] <- [$$d.getField(1)] project: [$$38] [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
+                    -- ASSIGN  |PARTITIONED|
+                      project ([$$d]) [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$38] <- [$$d.getField(1)] [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
-                        -- ASSIGN  |PARTITIONED|
-                          project ([$$d]) [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            exchange [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
+                        exchange [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$39, $$d] <- test.DBLP1 [cardinality: 100.0, op-cost: 100.0, total-cost: 100.0]
+                          -- DATASOURCE_SCAN  |PARTITIONED|
+                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              data-scan []<-[$$39, $$d] <- test.DBLP1 [cardinality: 100.0, op-cost: 100.0, total-cost: 100.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: 1.0, op-cost: 4.0, total-cost: 104.0]
-                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                      project ([$$41]) [cardinality: 1.0, op-cost: 0.0, total-cost: 100.0]
-                      -- STREAM_PROJECT  |UNPARTITIONED|
-                        assign [$$41] <- [get-item($$30, 0).getField(0).getField(1)] [cardinality: 1.0, op-cost: 0.0, total-cost: 100.0]
-                        -- ASSIGN  |UNPARTITIONED|
-                          aggregate [$$30] <- [listify($$29)] [cardinality: 1.0, op-cost: 0.0, total-cost: 100.0]
-                          -- AGGREGATE  |UNPARTITIONED|
-                            limit 1 [cardinality: 1.0, op-cost: 0.0, total-cost: 100.0]
-                            -- STREAM_LIMIT  |UNPARTITIONED|
-                              exchange [cardinality: 1.0, op-cost: 4.0, total-cost: 104.0]
-                              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                project ([$$29]) [cardinality: 1.0, op-cost: 0.0, total-cost: 100.0]
+                              empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  exchange [cardinality: 1.0, op-cost: 4.0, total-cost: 104.0]
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    assign [$$41] <- [get-item($$30, 0).getField(0).getField(1)] project: [$$41] [cardinality: 1.0, op-cost: 0.0, total-cost: 100.0]
+                    -- ASSIGN  |UNPARTITIONED|
+                      aggregate [$$30] <- [listify($$29)] [cardinality: 1.0, op-cost: 0.0, total-cost: 100.0]
+                      -- AGGREGATE  |UNPARTITIONED|
+                        limit 1 [cardinality: 1.0, op-cost: 0.0, total-cost: 100.0]
+                        -- STREAM_LIMIT  |UNPARTITIONED|
+                          exchange [cardinality: 1.0, op-cost: 4.0, total-cost: 104.0]
+                          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                            assign [$$29] <- [{"d": $$d}] project: [$$29] [cardinality: 1.0, op-cost: 0.0, total-cost: 100.0]
+                            -- ASSIGN  |PARTITIONED|
+                              limit 1 [cardinality: 1.0, op-cost: 0.0, total-cost: 100.0]
+                              -- STREAM_LIMIT  |PARTITIONED|
+                                project ([$$d]) [cardinality: 1.0, op-cost: 0.0, total-cost: 100.0]
                                 -- STREAM_PROJECT  |PARTITIONED|
-                                  assign [$$29] <- [{"d": $$d}] [cardinality: 1.0, op-cost: 0.0, total-cost: 100.0]
-                                  -- ASSIGN  |PARTITIONED|
-                                    limit 1 [cardinality: 1.0, op-cost: 0.0, total-cost: 100.0]
-                                    -- STREAM_LIMIT  |PARTITIONED|
-                                      project ([$$d]) [cardinality: 1.0, op-cost: 0.0, total-cost: 100.0]
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        exchange [cardinality: 1.0, op-cost: 4.0, total-cost: 104.0]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          data-scan []<-[$$40, $$d] <- test.DBLP1 condition (ends-with($$d.getField(1), "Blakeley95")) limit 1 [cardinality: 1.0, op-cost: 100.0, total-cost: 100.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: 1.0, op-cost: 4.0, total-cost: 104.0]
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    data-scan []<-[$$40, $$d] <- test.DBLP1 condition (ends-with($$d.getField(1), "Blakeley95")) limit 1 [cardinality: 1.0, op-cost: 100.0, total-cost: 100.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_cbo/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.6.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.6.plan
index 92cdd0c..8f65896 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.6.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.6.plan
@@ -6,23 +6,19 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 100.0]
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        project ([$$19]) [cardinality: 2.0, op-cost: 0.0, total-cost: 100.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          assign [$$19] <- [{"$1": substring($$20, 0, 21)}] [cardinality: 2.0, op-cost: 0.0, total-cost: 100.0]
-          -- ASSIGN  |PARTITIONED|
-            limit 1 [cardinality: 2.0, op-cost: 0.0, total-cost: 100.0]
-            -- STREAM_LIMIT  |PARTITIONED|
-              project ([$$20]) [cardinality: 2.0, op-cost: 0.0, total-cost: 100.0]
+        assign [$$19] <- [{"$1": substring($$20, 0, 21)}] project: [$$19] [cardinality: 2.0, op-cost: 0.0, total-cost: 100.0]
+        -- ASSIGN  |PARTITIONED|
+          limit 1 [cardinality: 2.0, op-cost: 0.0, total-cost: 100.0]
+          -- STREAM_LIMIT  |PARTITIONED|
+            assign [$$20] <- [$$DBLP1.getField(1)] project: [$$20] [cardinality: 2.0, op-cost: 0.0, total-cost: 100.0]
+            -- ASSIGN  |PARTITIONED|
+              project ([$$DBLP1]) [cardinality: 2.0, op-cost: 0.0, total-cost: 100.0]
               -- STREAM_PROJECT  |PARTITIONED|
-                assign [$$20] <- [$$DBLP1.getField(1)] [cardinality: 2.0, op-cost: 0.0, total-cost: 100.0]
-                -- ASSIGN  |PARTITIONED|
-                  project ([$$DBLP1]) [cardinality: 2.0, op-cost: 0.0, total-cost: 100.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 100.0]
+                exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 100.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$21, $$DBLP1] <- test.DBLP1 condition (gt($$DBLP1.getField(1), "series")) limit 1 [cardinality: 2.0, op-cost: 100.0, total-cost: 100.0]
+                  -- DATASOURCE_SCAN  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      data-scan []<-[$$21, $$DBLP1] <- test.DBLP1 condition (gt($$DBLP1.getField(1), "series")) limit 1 [cardinality: 2.0, op-cost: 100.0, total-cost: 100.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|
+                      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_cbo/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.plan
index d70573c..7f5d5d3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.plan
@@ -8,21 +8,17 @@
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
         limit 2 [cardinality: 3.0, op-cost: 0.0, total-cost: 12.0]
         -- STREAM_LIMIT  |PARTITIONED|
-          project ([$$22]) [cardinality: 3.0, op-cost: 0.0, total-cost: 12.0]
-          -- STREAM_PROJECT  |PARTITIONED|
-            assign [$$22] <- [$$26.getField("lang")] [cardinality: 3.0, op-cost: 0.0, total-cost: 12.0]
+          assign [$$22] <- [$$26.getField("lang")] project: [$$22] [cardinality: 3.0, op-cost: 0.0, total-cost: 12.0]
+          -- ASSIGN  |PARTITIONED|
+            assign [$$26] <- [$$t.getField("user")] project: [$$26] [cardinality: 3.0, op-cost: 0.0, total-cost: 12.0]
             -- ASSIGN  |PARTITIONED|
-              project ([$$26]) [cardinality: 3.0, op-cost: 0.0, total-cost: 12.0]
+              project ([$$t]) [cardinality: 3.0, op-cost: 0.0, total-cost: 12.0]
               -- STREAM_PROJECT  |PARTITIONED|
-                assign [$$26] <- [$$t.getField("user")] [cardinality: 3.0, op-cost: 0.0, total-cost: 12.0]
-                -- ASSIGN  |PARTITIONED|
-                  project ([$$t]) [cardinality: 3.0, op-cost: 0.0, total-cost: 12.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    exchange [cardinality: 3.0, op-cost: 0.0, total-cost: 12.0]
+                exchange [cardinality: 3.0, op-cost: 0.0, total-cost: 12.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  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: 3.0, op-cost: 12.0, total-cost: 12.0]
+                  -- DATASOURCE_SCAN  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      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: 3.0, op-cost: 12.0, total-cost: 12.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|
+                      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_cbo/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.8.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.8.plan
index f4b5d03..5024957 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.8.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.8.plan
@@ -4,40 +4,38 @@
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit 5 offset 5 [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$80]) [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
-      -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$80] <- [get-item($$78, 0)] [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
-        -- ASSIGN  |PARTITIONED|
-          project ([$$78]) [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
-          -- STREAM_PROJECT  |PARTITIONED|
-            exchange [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
-            -- SORT_MERGE_EXCHANGE [$$82(ASC) ]  |PARTITIONED|
-              limit 10 [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
-              -- STREAM_LIMIT  |PARTITIONED|
-                exchange [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  order (topK: 10) (ASC, $$82) [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
-                  -- STABLE_SORT [topK: 10] [$$82(ASC)]  |PARTITIONED|
-                    exchange [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      project ([$$78, $$82]) [cardinality: 100.0, op-cost: 0.0, total-cost: 100.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: 100.0, op-cost: 0.0, total-cost: 100.0]
-                        -- SUBPLAN  |PARTITIONED|
-                          exchange [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            data-scan []<-[$$82, $$paper] <- test.DBLP1 [cardinality: 100.0, op-cost: 100.0, total-cost: 100.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|
+      assign [$$80] <- [get-item($$78, 0)] project: [$$80] [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
+      -- ASSIGN  |PARTITIONED|
+        project ([$$78]) [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
+        -- STREAM_PROJECT  |PARTITIONED|
+          exchange [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
+          -- SORT_MERGE_EXCHANGE [$$82(ASC) ]  |PARTITIONED|
+            limit 10 [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
+            -- STREAM_LIMIT  |PARTITIONED|
+              exchange [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                order (topK: 10) (ASC, $$82) [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
+                -- STABLE_SORT [topK: 10] [$$82(ASC)]  |PARTITIONED|
+                  exchange [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    project ([$$78, $$82]) [cardinality: 100.0, op-cost: 0.0, total-cost: 100.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: 100.0, op-cost: 0.0, total-cost: 100.0]
+                      -- SUBPLAN  |PARTITIONED|
+                        exchange [cardinality: 100.0, op-cost: 0.0, total-cost: 100.0]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$82, $$paper] <- test.DBLP1 [cardinality: 100.0, op-cost: 100.0, total-cost: 100.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_cbo/objects/load-record-fields/load-record-fields.4.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/objects/load-record-fields/load-record-fields.4.plan
index 2ec98bb..ffb14f4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/objects/load-record-fields/load-record-fields.4.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/objects/load-record-fields/load-record-fields.4.plan
@@ -2,25 +2,23 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$30]) [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$30] <- [$$md.getField("name")] [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$md]) [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-          -- SORT_MERGE_EXCHANGE [$$32(ASC) ]  |PARTITIONED|
-            order (ASC, $$32) [cardinality: 2.0, op-cost: 2.0, total-cost: 4.0]
-            -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
-              exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                select (neq(uuid(), uuid())) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$32, $$md] <- test.MyDataset [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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|
+    assign [$$30] <- [$$md.getField("name")] project: [$$30] [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$md]) [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
+        -- SORT_MERGE_EXCHANGE [$$32(ASC) ]  |PARTITIONED|
+          order (ASC, $$32) [cardinality: 2.0, op-cost: 2.0, total-cost: 4.0]
+          -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+            exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              select (neq(uuid(), uuid())) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$32, $$md] <- test.MyDataset [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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_cbo/objects/load-record-fields/load-record-fields.6.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/objects/load-record-fields/load-record-fields.6.plan
index 32dbed9..4203276 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/objects/load-record-fields/load-record-fields.6.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/objects/load-record-fields/load-record-fields.6.plan
@@ -2,25 +2,23 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$30]) [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$30] <- [$$md.getField("name")] [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$md]) [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
-          -- SORT_MERGE_EXCHANGE [$$32(ASC) ]  |PARTITIONED|
-            order (ASC, $$32) [cardinality: 2.0, op-cost: 2.0, total-cost: 4.0]
-            -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
-              exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                select (neq(current-date(), date: { 1980-09-10 })) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$32, $$md] <- test.MyDataset [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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|
+    assign [$$30] <- [$$md.getField("name")] project: [$$30] [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$md]) [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 4.0]
+        -- SORT_MERGE_EXCHANGE [$$32(ASC) ]  |PARTITIONED|
+          order (ASC, $$32) [cardinality: 2.0, op-cost: 2.0, total-cost: 4.0]
+          -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+            exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              select (neq(current-date(), date: { 1980-09-10 })) [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 2.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$32, $$md] <- test.MyDataset [cardinality: 2.0, op-cost: 2.0, total-cost: 2.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_cbo/query_index/q01/q01.024.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/query_index/q01/q01.024.plan
index a00d7f6..f13eb9e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/query_index/q01/q01.024.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/query_index/q01/q01.024.plan
@@ -2,19 +2,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$v]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$v] <- [{"SK0": $$14, "PK0": $$15}] [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 [$$15(ASC) ]  |PARTITIONED|
-          order (ASC, $$15) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              data-scan []<-[$$14, $$15] <- test.ds1.ds1_age.query-index [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|
+    assign [$$v] <- [{"SK0": $$14, "PK0": $$15}] project: [$$v] [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 [$$15(ASC) ]  |PARTITIONED|
+        order (ASC, $$15) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            data-scan []<-[$$14, $$15] <- test.ds1.ds1_age.query-index [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_cbo/query_index/q01/q01.025.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/query_index/q01/q01.025.plan
index 631c16c..2baa9ac 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/query_index/q01/q01.025.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/query_index/q01/q01.025.plan
@@ -2,19 +2,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$v]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$v] <- [{"SK0": $$14, "SK1": $$15, "PK0": $$16}] [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 [$$16(ASC) ]  |PARTITIONED|
-          order (ASC, $$16) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              data-scan []<-[$$14, $$15, $$16] <- test.ds1.ds1_age_dept.query-index [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|
+    assign [$$v] <- [{"SK0": $$14, "SK1": $$15, "PK0": $$16}] project: [$$v] [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 [$$16(ASC) ]  |PARTITIONED|
+        order (ASC, $$16) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            data-scan []<-[$$14, $$15, $$16] <- test.ds1.ds1_age_dept.query-index [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_cbo/query_index/q01/q01.026.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/query_index/q01/q01.026.plan
index 9a21589..c8b01e8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/query_index/q01/q01.026.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/query_index/q01/q01.026.plan
@@ -2,37 +2,35 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$57]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$57] <- [{"age": $$SK0, "dept": $$SK1, "cnt": $$63}] [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 [$$SK1(ASC), $$SK0(ASC) ]  |PARTITIONED|
-          group by ([$$SK1 := $$65; $$SK0 := $$66]) decor ([]) {
-                    aggregate [$$63] <- [agg-sql-sum($$64)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$65, $$66]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$65, $$66]  |PARTITIONED|
-              group by ([$$65 := $$61; $$66 := $$60]) decor ([]) {
-                        aggregate [$$64] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$61, $$60]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$61, $$60]) [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 []<-[$$60, $$61, $$62] <- test.ds1.ds1_age_dept.query-index [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|
+    assign [$$57] <- [{"age": $$SK0, "dept": $$SK1, "cnt": $$63}] project: [$$57] [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 [$$SK1(ASC), $$SK0(ASC) ]  |PARTITIONED|
+        group by ([$$SK1 := $$65; $$SK0 := $$66]) decor ([]) {
+                  aggregate [$$63] <- [agg-sql-sum($$64)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$65, $$66]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$65, $$66]  |PARTITIONED|
+            group by ([$$65 := $$61; $$66 := $$60]) decor ([]) {
+                      aggregate [$$64] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$61, $$60]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                project ([$$61, $$60]) [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 []<-[$$60, $$61, $$62] <- test.ds1.ds1_age_dept.query-index [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_cbo/query_index/q01/q01.027.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/query_index/q01/q01.027.plan
index 2c28594..f82628d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/query_index/q01/q01.027.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/query_index/q01/q01.027.plan
@@ -2,21 +2,19 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$33]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$33] <- [{"cnt": $$36}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$36] <- [agg-sql-sum($$37)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$37] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                data-scan []<-[$$34, $$35] <- test.ds1.ds1_dept.query-index [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|
+    assign [$$33] <- [{"cnt": $$36}] project: [$$33] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$36] <- [agg-sql-sum($$37)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$37] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$34, $$35] <- test.ds1.ds1_dept.query-index [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_cbo/query_index/q01/q01.028.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/query_index/q01/q01.028.plan
index f171976..ee7953e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/query_index/q01/q01.028.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/query_index/q01/q01.028.plan
@@ -2,37 +2,35 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$45]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$45] <- [{"age": $$SK0, "cnt": $$49}] [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 [$$SK0(ASC) ]  |PARTITIONED|
-          group by ([$$SK0 := $$51]) decor ([]) {
-                    aggregate [$$49] <- [agg-sql-sum($$50)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$51]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$51]  |PARTITIONED|
-              group by ([$$51 := $$47]) decor ([]) {
-                        aggregate [$$50] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- PRE_CLUSTERED_GROUP_BY[$$47]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$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|
-                      data-scan []<-[$$47, $$48] <- test.ds1.ds1_age.query-index [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|
+    assign [$$45] <- [{"age": $$SK0, "cnt": $$49}] project: [$$45] [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 [$$SK0(ASC) ]  |PARTITIONED|
+        group by ([$$SK0 := $$51]) decor ([]) {
+                  aggregate [$$49] <- [agg-sql-sum($$50)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$51]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$51]  |PARTITIONED|
+            group by ([$$51 := $$47]) decor ([]) {
+                      aggregate [$$50] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- PRE_CLUSTERED_GROUP_BY[$$47]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                project ([$$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|
+                    data-scan []<-[$$47, $$48] <- test.ds1.ds1_age.query-index [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_cbo/query_index/q01/q01.029.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/query_index/q01/q01.029.plan
index ed25d3a..99731ea 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/query_index/q01/q01.029.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/query_index/q01/q01.029.plan
@@ -2,37 +2,35 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$45]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$45] <- [{"age": $$SK0, "cnt": $$50}] [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 [$$SK0(ASC) ]  |PARTITIONED|
-          group by ([$$SK0 := $$52]) decor ([]) {
-                    aggregate [$$50] <- [agg-sql-sum($$51)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$52]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
-              group by ([$$52 := $$47]) decor ([]) {
-                        aggregate [$$51] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- PRE_CLUSTERED_GROUP_BY[$$47]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$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|
-                      data-scan []<-[$$47, $$48, $$49] <- test.ds1.ds1_age_dept.query-index [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|
+    assign [$$45] <- [{"age": $$SK0, "cnt": $$50}] project: [$$45] [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 [$$SK0(ASC) ]  |PARTITIONED|
+        group by ([$$SK0 := $$52]) decor ([]) {
+                  aggregate [$$50] <- [agg-sql-sum($$51)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$52]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
+            group by ([$$52 := $$47]) decor ([]) {
+                      aggregate [$$51] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- PRE_CLUSTERED_GROUP_BY[$$47]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                project ([$$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|
+                    data-scan []<-[$$47, $$48, $$49] <- test.ds1.ds1_age_dept.query-index [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_cbo/query_index/q01/q01.030.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/query_index/q01/q01.030.plan
index 05dc8da..7c9818e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/query_index/q01/q01.030.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/query_index/q01/q01.030.plan
@@ -2,37 +2,35 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$45]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$45] <- [{"age": $$SK1, "cnt": $$50}] [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 [$$SK1(ASC) ]  |PARTITIONED|
-          group by ([$$SK1 := $$52]) decor ([]) {
-                    aggregate [$$50] <- [agg-sql-sum($$51)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$52]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
-              group by ([$$52 := $$48]) decor ([]) {
-                        aggregate [$$51] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$48]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$48]) [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 []<-[$$47, $$48, $$49] <- test.ds1.ds1_dept_age.query-index [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|
+    assign [$$45] <- [{"age": $$SK1, "cnt": $$50}] project: [$$45] [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 [$$SK1(ASC) ]  |PARTITIONED|
+        group by ([$$SK1 := $$52]) decor ([]) {
+                  aggregate [$$50] <- [agg-sql-sum($$51)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$52]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
+            group by ([$$52 := $$48]) decor ([]) {
+                      aggregate [$$51] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$48]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                project ([$$48]) [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 []<-[$$47, $$48, $$49] <- test.ds1.ds1_dept_age.query-index [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_cbo/query_index/q01/q01.031.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/query_index/q01/q01.031.plan
index 42f5940..8f01ff7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/query_index/q01/q01.031.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/query_index/q01/q01.031.plan
@@ -2,37 +2,35 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$57]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$57] <- [{"age": $$SK0, "dept": $$SK1, "cnt": $$63}] [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 [$$SK0(ASC), $$SK1(ASC) ]  |PARTITIONED|
-          group by ([$$SK0 := $$65; $$SK1 := $$66]) decor ([]) {
-                    aggregate [$$63] <- [agg-sql-sum($$64)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$65, $$66]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$65, $$66]  |PARTITIONED|
-              group by ([$$65 := $$60; $$66 := $$61]) decor ([]) {
-                        aggregate [$$64] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- PRE_CLUSTERED_GROUP_BY[$$60, $$61]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$60, $$61]) [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 []<-[$$60, $$61, $$62] <- test.ds1.ds1_age_dept.query-index [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|
+    assign [$$57] <- [{"age": $$SK0, "dept": $$SK1, "cnt": $$63}] project: [$$57] [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 [$$SK0(ASC), $$SK1(ASC) ]  |PARTITIONED|
+        group by ([$$SK0 := $$65; $$SK1 := $$66]) decor ([]) {
+                  aggregate [$$63] <- [agg-sql-sum($$64)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$65, $$66]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$65, $$66]  |PARTITIONED|
+            group by ([$$65 := $$60; $$66 := $$61]) decor ([]) {
+                      aggregate [$$64] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- PRE_CLUSTERED_GROUP_BY[$$60, $$61]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                project ([$$60, $$61]) [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 []<-[$$60, $$61, $$62] <- test.ds1.ds1_age_dept.query-index [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_cbo/subquery/query-ASTERIXDB-3538/query-ASTERIXDB-3538.3.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/subquery/query-ASTERIXDB-3538/query-ASTERIXDB-3538.3.plan
index dc43c42..8d2d305 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/subquery/query-ASTERIXDB-3538/query-ASTERIXDB-3538.3.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/subquery/query-ASTERIXDB-3538/query-ASTERIXDB-3538.3.plan
@@ -2,23 +2,19 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$311]) [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$311] <- [{"id": $$345}] [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
-      -- ASSIGN  |PARTITIONED|
-        select (not(is-null($$345))) [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
-        -- STREAM_SELECT  |PARTITIONED|
-          project ([$$345]) [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
+    assign [$$311] <- [{"id": $$345}] project: [$$311] [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
+    -- ASSIGN  |PARTITIONED|
+      select (not(is-null($$345))) [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        assign [$$345] <- [string-default-null($$s.getField("id"))] project: [$$345] [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
+        -- ASSIGN  |PARTITIONED|
+          project ([$$s]) [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
           -- STREAM_PROJECT  |PARTITIONED|
-            assign [$$345] <- [string-default-null($$s.getField("id"))] [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
-            -- ASSIGN  |PARTITIONED|
-              project ([$$s]) [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                exchange [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
+            exchange [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$314, $$s] <- test.dat1 [cardinality: 8.0, op-cost: 8.0, total-cost: 8.0]
+              -- DATASOURCE_SCAN  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  data-scan []<-[$$314, $$s] <- test.dat1 [cardinality: 8.0, op-cost: 8.0, total-cost: 8.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|
+                  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_cbo/subquery/query-ASTERIXDB-3538/query-ASTERIXDB-3538.5.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/subquery/query-ASTERIXDB-3538/query-ASTERIXDB-3538.5.plan
index b9dda19..efa194b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/subquery/query-ASTERIXDB-3538/query-ASTERIXDB-3538.5.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/subquery/query-ASTERIXDB-3538/query-ASTERIXDB-3538.5.plan
@@ -2,40 +2,34 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 4.0, op-cost: 0.0, total-cost: 8.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$317]) [cardinality: 4.0, op-cost: 0.0, total-cost: 8.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$317] <- [{"dat3": {"id": $$351, "a": string-default-null($$345), "d": int64-default-null($$318)}}] [cardinality: 4.0, op-cost: 0.0, total-cost: 8.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$351, $$345, $$318]) [cardinality: 4.0, op-cost: 0.0, total-cost: 8.0]
+    assign [$$317] <- [{"dat3": {"id": $$351, "a": string-default-null($$345), "d": int64-default-null($$318)}}] project: [$$317] [cardinality: 4.0, op-cost: 0.0, total-cost: 8.0]
+    -- ASSIGN  |PARTITIONED|
+      select (le($$318, get-item($$295, 0))) project: [$$351, $$345, $$318] [cardinality: 4.0, op-cost: 0.0, total-cost: 8.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$351, $$345, $$318, $$295]) [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
         -- STREAM_PROJECT  |PARTITIONED|
-          select (le($$318, get-item($$295, 0))) [cardinality: 4.0, op-cost: 0.0, total-cost: 8.0]
-          -- STREAM_SELECT  |PARTITIONED|
-            project ([$$351, $$345, $$318, $$295]) [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              subplan {
-                        aggregate [$$295] <- [listify($$330)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          aggregate [$$330] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            unnest $$333 <- scan-collection($$343) [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: 8.0, op-cost: 0.0, total-cost: 8.0]
-              -- SUBPLAN  |PARTITIONED|
-                select (not(is-null($$351))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  project ([$$351, $$345, $$318, $$343]) [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$351, $$345, $$318, $$343] <- [string-default-null($$s.getField("id")), $$s.getField("a"), $$s.getField("d"), $$s.getField("e")] [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
-                    -- ASSIGN  |PARTITIONED|
-                      project ([$$s]) [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        exchange [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$319, $$s] <- test.dat1 [cardinality: 8.0, op-cost: 8.0, total-cost: 8.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|
+          subplan {
+                    aggregate [$$295] <- [listify($$330)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      aggregate [$$330] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        unnest $$333 <- scan-collection($$343) [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: 8.0, op-cost: 0.0, total-cost: 8.0]
+          -- SUBPLAN  |PARTITIONED|
+            select (not(is-null($$351))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$351, $$345, $$318, $$343] <- [string-default-null($$s.getField("id")), $$s.getField("a"), $$s.getField("d"), $$s.getField("e")] project: [$$351, $$345, $$318, $$343] [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
+              -- ASSIGN  |PARTITIONED|
+                project ([$$s]) [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
+                -- STREAM_PROJECT  |PARTITIONED|
+                  exchange [cardinality: 8.0, op-cost: 0.0, total-cost: 8.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$319, $$s] <- test.dat1 [cardinality: 8.0, op-cost: 8.0, total-cost: 8.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_cbo/tpch/query-plans/query-plans.03.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.03.plan
index 3f37bf4..3645f6b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.03.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.03.plan
@@ -2,47 +2,43 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 4.0, op-cost: 0.0, total-cost: 1794944.26]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$134]) [cardinality: 4.0, op-cost: 0.0, total-cost: 1794944.26]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$134] <- [{"$1": 1, "l_returnflag": $$l_returnflag, "l_linestatus": $$l_linestatus, "sum_qty": $$140, "sum_base_price": $$141, "sum_disc_price": $$142, "sum_charge": $$143, "avg_qty": $$144, "avg_price": $$145, "avg_disc": $$146, "count_order": $$147}] [cardinality: 4.0, op-cost: 0.0, total-cost: 1794944.26]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 4.0, op-cost: 0.0, total-cost: 1794944.26]
-        -- SORT_MERGE_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC) ]  |PARTITIONED|
-          order (ASC, $$l_returnflag) (ASC, $$l_linestatus) [cardinality: 4.0, op-cost: 8.0, total-cost: 1794944.26]
-          -- STABLE_SORT [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
-            exchange [cardinality: 4.0, op-cost: 0.0, total-cost: 1794936.26]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              group by ([$$l_returnflag := $$160; $$l_linestatus := $$161]) decor ([]) {
-                        aggregate [$$140, $$141, $$142, $$143, $$144, $$145, $$146, $$147] <- [global-sql-sum-serial($$152), global-sql-sum-serial($$153), global-sql-sum-serial($$154), global-sql-sum-serial($$155), global-sql-avg-serial($$156), global-sql-avg-serial($$157), global-sql-avg-serial($$158), sql-sum-serial($$159)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 4.0, op-cost: 597182.13, total-cost: 1794936.26]
-              -- EXTERNAL_GROUP_BY[$$160, $$161]  |PARTITIONED|
-                exchange [cardinality: 4.0, op-cost: 0.0, total-cost: 1197754.13]
-                -- HASH_PARTITION_EXCHANGE [$$160, $$161]  |PARTITIONED|
-                  group by ([$$160 := $$135; $$161 := $$136]) decor ([]) {
-                            aggregate [$$152, $$153, $$154, $$155, $$156, $$157, $$158, $$159] <- [local-sql-sum-serial($$88), local-sql-sum-serial($$93), local-sql-sum-serial(numeric-multiply($$93, numeric-subtract(1, $$149))), local-sql-sum-serial(numeric-multiply(numeric-multiply($$93, numeric-subtract(1, $$149)), numeric-add(1, $$151))), local-sql-avg-serial($$88), local-sql-avg-serial($$93), local-sql-avg-serial($$149), sql-count-serial(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- AGGREGATE  |LOCAL|
-                              nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                         } [cardinality: 4.0, op-cost: 597182.13, total-cost: 1197754.13]
-                  -- EXTERNAL_GROUP_BY[$$135, $$136]  |PARTITIONED|
-                    exchange [cardinality: 597182.13, op-cost: 0.0, total-cost: 600572.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      project ([$$88, $$93, $$149, $$151, $$135, $$136]) [cardinality: 597182.13, op-cost: 0.0, total-cost: 600572.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        select (le($$l.getField(10), "1998-09-16")) [cardinality: 597182.13, op-cost: 0.0, total-cost: 600572.0]
-                        -- STREAM_SELECT  |PARTITIONED|
-                          assign [$$136, $$135, $$88, $$149, $$151, $$93] <- [$$l.getField(9), $$l.getField(8), $$l.getField(4), $$l.getField(6), $$l.getField(7), $$l.getField(5)] [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
-                          -- ASSIGN  |PARTITIONED|
-                            project ([$$l]) [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              exchange [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
+    assign [$$134] <- [{"$1": 1, "l_returnflag": $$l_returnflag, "l_linestatus": $$l_linestatus, "sum_qty": $$140, "sum_base_price": $$141, "sum_disc_price": $$142, "sum_charge": $$143, "avg_qty": $$144, "avg_price": $$145, "avg_disc": $$146, "count_order": $$147}] project: [$$134] [cardinality: 4.0, op-cost: 0.0, total-cost: 1794944.26]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 4.0, op-cost: 0.0, total-cost: 1794944.26]
+      -- SORT_MERGE_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC) ]  |PARTITIONED|
+        order (ASC, $$l_returnflag) (ASC, $$l_linestatus) [cardinality: 4.0, op-cost: 8.0, total-cost: 1794944.26]
+        -- STABLE_SORT [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
+          exchange [cardinality: 4.0, op-cost: 0.0, total-cost: 1794936.26]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            group by ([$$l_returnflag := $$160; $$l_linestatus := $$161]) decor ([]) {
+                      aggregate [$$140, $$141, $$142, $$143, $$144, $$145, $$146, $$147] <- [global-sql-sum-serial($$152), global-sql-sum-serial($$153), global-sql-sum-serial($$154), global-sql-sum-serial($$155), global-sql-avg-serial($$156), global-sql-avg-serial($$157), global-sql-avg-serial($$158), sql-sum-serial($$159)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 4.0, op-cost: 597182.13, total-cost: 1794936.26]
+            -- EXTERNAL_GROUP_BY[$$160, $$161]  |PARTITIONED|
+              exchange [cardinality: 4.0, op-cost: 0.0, total-cost: 1197754.13]
+              -- HASH_PARTITION_EXCHANGE [$$160, $$161]  |PARTITIONED|
+                group by ([$$160 := $$135; $$161 := $$136]) decor ([]) {
+                          aggregate [$$152, $$153, $$154, $$155, $$156, $$157, $$158, $$159] <- [local-sql-sum-serial($$88), local-sql-sum-serial($$93), local-sql-sum-serial(numeric-multiply($$93, numeric-subtract(1, $$149))), local-sql-sum-serial(numeric-multiply(numeric-multiply($$93, numeric-subtract(1, $$149)), numeric-add(1, $$151))), local-sql-avg-serial($$88), local-sql-avg-serial($$93), local-sql-avg-serial($$149), sql-count-serial(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- AGGREGATE  |LOCAL|
+                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                       } [cardinality: 4.0, op-cost: 597182.13, total-cost: 1197754.13]
+                -- EXTERNAL_GROUP_BY[$$135, $$136]  |PARTITIONED|
+                  exchange [cardinality: 597182.13, op-cost: 0.0, total-cost: 600572.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    select (le($$l.getField(10), "1998-09-16")) project: [$$88, $$93, $$149, $$151, $$135, $$136] [cardinality: 597182.13, op-cost: 0.0, total-cost: 600572.0]
+                    -- STREAM_SELECT  |PARTITIONED|
+                      assign [$$136, $$135, $$88, $$149, $$151, $$93] <- [$$l.getField(9), $$l.getField(8), $$l.getField(4), $$l.getField(6), $$l.getField(7), $$l.getField(5)] [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
+                      -- ASSIGN  |PARTITIONED|
+                        project ([$$l]) [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          exchange [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            data-scan []<-[$$138, $$139, $$l] <- tpch.lineitem [cardinality: 600572.0, op-cost: 600572.0, total-cost: 600572.0]
+                            -- DATASOURCE_SCAN  |PARTITIONED|
+                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                data-scan []<-[$$138, $$139, $$l] <- tpch.lineitem [cardinality: 600572.0, op-cost: 600572.0, total-cost: 600572.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|
+                                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_cbo/tpch/query-plans/query-plans.04.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.04.plan
index 8b34297..66fe341 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.04.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.04.plan
@@ -4,290 +4,274 @@
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit 100 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$194]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$194] <- [{"$1": 2, "s_acctbal": $$237, "s_name": $$239, "n_name": $$238, "p_partkey": $$250, "p_mfgr": $$244, "s_address": $$245, "s_phone": $$246, "s_comment": $$247}] [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 [$$237(DESC), $$238(ASC), $$239(ASC), $$250(ASC) ]  |PARTITIONED|
-            limit 100 [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]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (topK: 100) (DESC, $$237) (ASC, $$238) (ASC, $$239) (ASC, $$250) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STABLE_SORT [topK: 100] [$$237(DESC), $$238(ASC), $$239(ASC), $$250(ASC)]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$237, $$239, $$238, $$250, $$244, $$245, $$246, $$247]) [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|
-                        join (eq($$221, $$206)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HYBRID_HASH_JOIN [$$221][$$206]  |PARTITIONED|
-                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- HASH_PARTITION_EXCHANGE [$$221]  |PARTITIONED|
-                            project ([$$237, $$239, $$238, $$250, $$244, $$245, $$246, $$247, $$221]) [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|
-                                join (eq($$225, $$205)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- HYBRID_HASH_JOIN [$$225][$$205]  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- HASH_PARTITION_EXCHANGE [$$225]  |PARTITIONED|
-                                    project ([$$237, $$239, $$250, $$244, $$245, $$246, $$247, $$225]) [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|
-                                        join (and(eq($$250, $$203), eq($$202, $$231))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- HYBRID_HASH_JOIN [$$250, $$231][$$203, $$202]  |PARTITIONED|
-                                          exchange [cardinality: 33.96, op-cost: 0.0, total-cost: 402536.81]
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            project ([$$250, $$244, $$231]) [cardinality: 33.96, op-cost: 0.0, total-cost: 402536.81]
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              assign [$$231] <- [get-item($$181, 0)] [cardinality: 33.96, op-cost: 0.0, total-cost: 402536.81]
-                                              -- ASSIGN  |PARTITIONED|
-                                                exchange [cardinality: 33.96, op-cost: 0.0, total-cost: 402536.81]
+      assign [$$194] <- [{"$1": 2, "s_acctbal": $$237, "s_name": $$239, "n_name": $$238, "p_partkey": $$250, "p_mfgr": $$244, "s_address": $$245, "s_phone": $$246, "s_comment": $$247}] project: [$$194] [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 [$$237(DESC), $$238(ASC), $$239(ASC), $$250(ASC) ]  |PARTITIONED|
+          limit 100 [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]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              order (topK: 100) (DESC, $$237) (ASC, $$238) (ASC, $$239) (ASC, $$250) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STABLE_SORT [topK: 100] [$$237(DESC), $$238(ASC), $$239(ASC), $$250(ASC)]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  project ([$$237, $$239, $$238, $$250, $$244, $$245, $$246, $$247]) [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|
+                      join (eq($$221, $$206)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- HYBRID_HASH_JOIN [$$221][$$206]  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- HASH_PARTITION_EXCHANGE [$$221]  |PARTITIONED|
+                          project ([$$237, $$239, $$238, $$250, $$244, $$245, $$246, $$247, $$221]) [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|
+                              join (eq($$225, $$205)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- HYBRID_HASH_JOIN [$$225][$$205]  |PARTITIONED|
+                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- HASH_PARTITION_EXCHANGE [$$225]  |PARTITIONED|
+                                  project ([$$237, $$239, $$250, $$244, $$245, $$246, $$247, $$225]) [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|
+                                      join (and(eq($$250, $$203), eq($$202, $$231))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- HYBRID_HASH_JOIN [$$250, $$231][$$203, $$202]  |PARTITIONED|
+                                        exchange [cardinality: 37.63, op-cost: 0.0, total-cost: 20000.0]
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          assign [$$231] <- [get-item($$181, 0)] project: [$$250, $$244, $$231] [cardinality: 37.63, op-cost: 0.0, total-cost: 20000.0]
+                                          -- ASSIGN  |PARTITIONED|
+                                            exchange [cardinality: 37.63, op-cost: 0.0, total-cost: 20000.0]
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              group by ([$$250 := $$200]) decor ([$$244]) {
+                                                        aggregate [$$181] <- [listify($$215)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                        -- AGGREGATE  |LOCAL|
+                                                          aggregate [$$215] <- [agg-sql-min($$208)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                          -- AGGREGATE  |LOCAL|
+                                                            select (not(is-missing($$249))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                            -- STREAM_SELECT  |LOCAL|
+                                                              nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 20000.0]
+                                              -- PRE_CLUSTERED_GROUP_BY[$$200]  |PARTITIONED|
+                                                exchange [cardinality: 37.63, op-cost: 0.0, total-cost: 20000.0]
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  group by ([$$250 := $$200]) decor ([$$244]) {
-                                                            aggregate [$$181] <- [listify($$215)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                            -- AGGREGATE  |LOCAL|
-                                                              aggregate [$$215] <- [agg-sql-min($$208)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                              -- AGGREGATE  |LOCAL|
-                                                                nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                         } [cardinality: 0.0, op-cost: 0.0, total-cost: 402536.81]
-                                                  -- PRE_CLUSTERED_GROUP_BY[$$200]  |PARTITIONED|
-                                                    exchange [cardinality: 33.96, op-cost: 0.0, total-cost: 402536.81]
+                                                  order (ASC, $$200) [cardinality: 37.63, op-cost: 0.0, total-cost: 20000.0]
+                                                  -- STABLE_SORT [$$200(ASC)]  |PARTITIONED|
+                                                    exchange [cardinality: 37.63, op-cost: 0.0, total-cost: 20000.0]
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      order (ASC, $$200) [cardinality: 33.96, op-cost: 0.0, total-cost: 402536.81]
-                                                      -- STABLE_SORT [$$200(ASC)]  |PARTITIONED|
-                                                        exchange [cardinality: 33.96, op-cost: 0.0, total-cost: 402536.81]
-                                                        -- HASH_PARTITION_EXCHANGE [$$200]  |PARTITIONED|
-                                                          project ([$$244, $$208, $$200]) [cardinality: 33.96, op-cost: 0.0, total-cost: 402536.81]
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            exchange [cardinality: 33.96, op-cost: 0.0, total-cost: 402536.81]
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              join (eq($$218, $$213)) [cardinality: 33.96, op-cost: 156.81, total-cost: 402536.81]
-                                                              -- HYBRID_HASH_JOIN [$$218][$$213]  |PARTITIONED|
-                                                                exchange [cardinality: 152.81, op-cost: 0.0, total-cost: 402371.01]
+                                                      project ([$$244, $$208, $$249, $$200]) [cardinality: 37.63, op-cost: 0.0, total-cost: 20000.0]
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        exchange [cardinality: 37.63, op-cost: 0.0, total-cost: 20000.0]
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          left outer join (eq($$207, $$200)) [cardinality: 30.16, op-cost: 16067.92, total-cost: 394475.1]
+                                                          -- HYBRID_HASH_JOIN [$$200][$$207]  |PARTITIONED|
+                                                            exchange [cardinality: 37.63, op-cost: 0.0, total-cost: 20000.0]
+                                                            -- HASH_PARTITION_EXCHANGE [$$200]  |PARTITIONED|
+                                                              select (and(eq($$p.getField(5), 37), like($$p.getField(4), "%COPPER"))) project: [$$244, $$200] [cardinality: 37.63, op-cost: 0.0, total-cost: 20000.0]
+                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                assign [$$244] <- [$$p.getField(2)] [cardinality: 20000.0, op-cost: 0.0, total-cost: 20000.0]
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  exchange [cardinality: 20000.0, op-cost: 0.0, total-cost: 20000.0]
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    data-scan []<-[$$200, $$p] <- tpch.part [cardinality: 20000.0, op-cost: 20000.0, total-cost: 20000.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 [$$207]  |PARTITIONED|
+                                                              project ([$$208, $$249, $$207]) [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|
-                                                                  project ([$$244, $$208, $$200, $$218]) [cardinality: 152.81, op-cost: 0.0, total-cost: 402371.01]
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    exchange [cardinality: 152.81, op-cost: 0.0, total-cost: 402371.01]
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      join (eq($$223, $$212)) [cardinality: 152.81, op-cost: 173.99, total-cost: 402371.01]
-                                                                      -- HYBRID_HASH_JOIN [$$223][$$212]  |PARTITIONED|
-                                                                        exchange [cardinality: 987.0, op-cost: 148.99, total-cost: 1148.99]
-                                                                        -- HASH_PARTITION_EXCHANGE [$$223]  |PARTITIONED|
-                                                                          project ([$$244, $$208, $$200, $$223]) [cardinality: 987.0, op-cost: 0.0, total-cost: 1000.0]
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            assign [$$223] <- [$$s2.getField(3)] [cardinality: 987.0, op-cost: 0.0, total-cost: 1000.0]
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              project ([$$208, $$244, $$200, $$s2]) [cardinality: 987.0, op-cost: 0.0, total-cost: 1000.0]
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                exchange [cardinality: 987.0, op-cost: 148.99, total-cost: 1148.99]
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  unnest-map [$$211, $$s2] <- index-search("supplier", 0, "Default", "tpch", "supplier", true, true, 1, $$210, 1, $$210, true, true, true) [cardinality: 987.0, op-cost: 1000.0, total-cost: 1000.0]
-                                                                                  -- BTREE_SEARCH  |PARTITIONED|
-                                                                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      order (ASC, $$210) [cardinality: 987.0, op-cost: 1000.0, total-cost: 1000.0]
-                                                                                      -- STABLE_SORT [$$210(ASC)]  |PARTITIONED|
-                                                                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                        -- HASH_PARTITION_EXCHANGE [$$210]  |PARTITIONED|
-                                                                                          project ([$$208, $$210, $$244, $$200]) [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 (eq($$207, $$200)) [cardinality: 150.8, op-cost: 80301.99, total-cost: 400640.65]
-                                                                                              -- HYBRID_HASH_JOIN [$$207][$$200]  |PARTITIONED|
-                                                                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                  project ([$$208, $$210, $$207]) [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|
-                                                                                                      join (eq($$207, $$209)) [cardinality: 80151.47, op-cost: 100094.07, total-cost: 300188.15]
-                                                                                                      -- HYBRID_HASH_JOIN [$$209][$$207]  |PARTITIONED|
-                                                                                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                        -- HASH_PARTITION_EXCHANGE [$$209]  |PARTITIONED|
-                                                                                                          replicate [cardinality: 80000.0, op-cost: 0.0, total-cost: 80000.0]
-                                                                                                          -- REPLICATE  |PARTITIONED|
-                                                                                                            exchange [cardinality: 80000.0, op-cost: 0.0, total-cost: 80000.0]
-                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                              project ([$$208, $$210, $$209]) [cardinality: 80000.0, op-cost: 0.0, total-cost: 80000.0]
-                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                exchange [cardinality: 80000.0, op-cost: 0.0, total-cost: 80000.0]
-                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                  data-scan []<-[$$208, $$209, $$210, $$ps2] <- tpch.partsupp [cardinality: 80000.0, op-cost: 80000.0, total-cost: 80000.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 [$$207]  |PARTITIONED|
-                                                                                                          project ([$$207]) [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|
-                                                                                                              replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                              -- REPLICATE  |PARTITIONED|
-                                                                                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                  data-scan []<-[$$207, $$p2] <- tpch.part [cardinality: 20000.0, op-cost: 20000.0, total-cost: 20000.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]
-                                                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                  project ([$$244, $$200]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                    select (and(eq($$p.getField(5), 37), like($$p.getField(4), "%COPPER"))) [cardinality: 37.63, op-cost: 0.0, total-cost: 20000.0]
-                                                                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                                                                      assign [$$244] <- [$$p.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                                        project ([$$200, $$p]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                          assign [$$200, $$p] <- [$$207, $$p2] [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]
-                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                              replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                              -- REPLICATE  |PARTITIONED|
-                                                                                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                  data-scan []<-[$$207, $$p2] <- tpch.part [cardinality: 20000.0, op-cost: 20000.0, total-cost: 20000.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: 25.0, op-cost: 25.0, total-cost: 50.0]
-                                                                        -- HASH_PARTITION_EXCHANGE [$$212]  |PARTITIONED|
-                                                                          project ([$$218, $$212]) [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            assign [$$218] <- [$$n2.getField(2)] [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
-                                                                            -- ASSIGN  |PARTITIONED|
-                                                                              exchange [cardinality: 25.0, op-cost: 25.0, total-cost: 50.0]
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                replicate [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
-                                                                                -- REPLICATE  |PARTITIONED|
-                                                                                  exchange [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
-                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    data-scan []<-[$$212, $$n2] <- tpch.nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.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: 1.0, op-cost: 4.0, total-cost: 9.0]
-                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                  replicate [cardinality: 1.0, op-cost: 0.0, total-cost: 5.0]
-                                                                  -- REPLICATE  |PARTITIONED|
-                                                                    exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 5.0]
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      project ([$$213]) [cardinality: 1.0, op-cost: 0.0, total-cost: 5.0]
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        select (eq($$r2.getField(1), "EUROPE")) [cardinality: 1.0, op-cost: 0.0, total-cost: 5.0]
-                                                                        -- STREAM_SELECT  |PARTITIONED|
-                                                                          exchange [cardinality: 5.0, op-cost: 0.0, total-cost: 5.0]
+                                                                  join (eq($$207, $$209)) [cardinality: 16030.29, op-cost: 36033.87, total-cost: 342339.25]
+                                                                  -- HYBRID_HASH_JOIN [$$209][$$207]  |PARTITIONED|
+                                                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                    -- HASH_PARTITION_EXCHANGE [$$209]  |PARTITIONED|
+                                                                      assign [$$249] <- [true] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        project ([$$208, $$209]) [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 []<-[$$213, $$r2] <- tpch.region [cardinality: 5.0, op-cost: 5.0, total-cost: 5.0]
-                                                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                            join (eq($$218, $$213)) [cardinality: 16015.05, op-cost: 80079.26, total-cost: 250271.52]
+                                                                            -- HYBRID_HASH_JOIN [$$218][$$213]  |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: 79109.84, op-cost: 0.0, total-cost: 168971.26]
-                                          -- HASH_PARTITION_EXCHANGE [$$203]  |PARTITIONED|
-                                            project ([$$237, $$239, $$245, $$246, $$247, $$225, $$203, $$202]) [cardinality: 79109.84, op-cost: 0.0, total-cost: 168971.26]
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              exchange [cardinality: 79109.84, op-cost: 0.0, total-cost: 168971.26]
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                join (eq($$201, $$204)) [cardinality: 79109.84, op-cost: 84023.26, total-cost: 168971.26]
-                                                -- HYBRID_HASH_JOIN [$$204][$$201]  |PARTITIONED|
-                                                  exchange [cardinality: 80000.0, op-cost: 0.0, total-cost: 80000.0]
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    project ([$$202, $$204, $$203]) [cardinality: 80000.0, op-cost: 0.0, total-cost: 80000.0]
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      assign [$$202, $$204, $$203] <- [$$208, $$210, $$209] [cardinality: 80000.0, op-cost: 0.0, total-cost: 80000.0]
-                                                      -- ASSIGN  |PARTITIONED|
+                                                                                project ([$$208, $$209, $$218]) [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|
+                                                                                    join (eq($$211, $$210)) [cardinality: 80075.26, op-cost: 84023.26, total-cost: 170183.26]
+                                                                                    -- HYBRID_HASH_JOIN [$$210][$$211]  |PARTITIONED|
+                                                                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        replicate [cardinality: 80000.0, op-cost: 0.0, total-cost: 80000.0]
+                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                          exchange [cardinality: 80000.0, op-cost: 0.0, total-cost: 80000.0]
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            project ([$$208, $$209, $$210]) [cardinality: 80000.0, op-cost: 0.0, total-cost: 80000.0]
+                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                              exchange [cardinality: 80000.0, op-cost: 0.0, total-cost: 80000.0]
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                data-scan []<-[$$208, $$209, $$210, $$ps2] <- tpch.partsupp [cardinality: 80000.0, op-cost: 80000.0, total-cost: 80000.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]
+                                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                        project ([$$218, $$211]) [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|
+                                                                                            join (eq($$223, $$212)) [cardinality: 987.0, op-cost: 1087.0, total-cost: 2212.0]
+                                                                                            -- HYBRID_HASH_JOIN [$$223][$$212]  |PARTITIONED|
+                                                                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                assign [$$223] <- [$$s2.getField(3)] project: [$$211, $$223] [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]
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    replicate [cardinality: 1000.0, op-cost: 0.0, total-cost: 1000.0]
+                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                      exchange [cardinality: 1000.0, op-cost: 3948.0, total-cost: 4948.0]
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        data-scan []<-[$$211, $$s2] <- tpch.supplier [cardinality: 1000.0, op-cost: 1000.0, total-cost: 1000.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]
+                                                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                assign [$$218] <- [$$n2.getField(2)] project: [$$218, $$212] [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]
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    replicate [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
+                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                      exchange [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        data-scan []<-[$$212, $$n2] <- tpch.nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.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]
+                                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                replicate [cardinality: 1.0, op-cost: 0.0, total-cost: 5.0]
+                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                  exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 5.0]
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    select (eq($$r2.getField(1), "EUROPE")) project: [$$213] [cardinality: 1.0, op-cost: 0.0, total-cost: 5.0]
+                                                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                                                      exchange [cardinality: 5.0, op-cost: 0.0, total-cost: 5.0]
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        data-scan []<-[$$213, $$r2] <- tpch.region [cardinality: 5.0, op-cost: 5.0, total-cost: 5.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 [$$207]  |PARTITIONED|
+                                                                      project ([$$207]) [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 []<-[$$207, $$p2] <- tpch.part [cardinality: 20000.0, op-cost: 20000.0, total-cost: 20000.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: 80075.26, op-cost: 0.0, total-cost: 168971.26]
+                                        -- HASH_PARTITION_EXCHANGE [$$203]  |PARTITIONED|
+                                          project ([$$237, $$239, $$245, $$246, $$247, $$225, $$203, $$202]) [cardinality: 80075.26, op-cost: 0.0, total-cost: 168971.26]
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            exchange [cardinality: 80075.26, op-cost: 0.0, total-cost: 168971.26]
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              join (eq($$201, $$204)) [cardinality: 80075.26, op-cost: 84023.26, total-cost: 168971.26]
+                                              -- HYBRID_HASH_JOIN [$$204][$$201]  |PARTITIONED|
+                                                exchange [cardinality: 80000.0, op-cost: 0.0, total-cost: 80000.0]
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  assign [$$202, $$203, $$204] <- [$$208, $$209, $$210] project: [$$202, $$203, $$204] [cardinality: 80000.0, op-cost: 0.0, total-cost: 80000.0]
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    exchange [cardinality: 80000.0, op-cost: 0.0, total-cost: 80000.0]
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      replicate [cardinality: 80000.0, op-cost: 0.0, total-cost: 80000.0]
+                                                      -- REPLICATE  |PARTITIONED|
                                                         exchange [cardinality: 80000.0, op-cost: 0.0, total-cost: 80000.0]
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          replicate [cardinality: 80000.0, op-cost: 0.0, total-cost: 80000.0]
-                                                          -- REPLICATE  |PARTITIONED|
+                                                          project ([$$208, $$209, $$210]) [cardinality: 80000.0, op-cost: 0.0, total-cost: 80000.0]
+                                                          -- STREAM_PROJECT  |PARTITIONED|
                                                             exchange [cardinality: 80000.0, op-cost: 0.0, total-cost: 80000.0]
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              project ([$$208, $$210, $$209]) [cardinality: 80000.0, op-cost: 0.0, total-cost: 80000.0]
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                exchange [cardinality: 80000.0, op-cost: 0.0, total-cost: 80000.0]
+                                                              data-scan []<-[$$208, $$209, $$210, $$ps2] <- tpch.partsupp [cardinality: 80000.0, op-cost: 80000.0, total-cost: 80000.0]
+                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  data-scan []<-[$$208, $$209, $$210, $$ps2] <- tpch.partsupp [cardinality: 80000.0, op-cost: 80000.0, total-cost: 80000.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: 1000.0, op-cost: 3948.0, total-cost: 4948.0]
-                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                    project ([$$237, $$239, $$245, $$246, $$247, $$225, $$201]) [cardinality: 1000.0, op-cost: 0.0, total-cost: 1000.0]
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      assign [$$247, $$246, $$245, $$237, $$239, $$225] <- [$$s.getField(6), $$s.getField(4), $$s.getField(2), $$s.getField(5), $$s.getField(1), $$s.getField(3)] [cardinality: 1000.0, op-cost: 0.0, total-cost: 1000.0]
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        exchange [cardinality: 1000.0, op-cost: 3948.0, total-cost: 4948.0]
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          data-scan []<-[$$201, $$s] <- tpch.supplier [cardinality: 1000.0, op-cost: 1000.0, total-cost: 1000.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: 25.0, op-cost: 0.0, total-cost: 25.0]
-                                  -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
-                                    project ([$$238, $$221, $$205]) [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      assign [$$238, $$221] <- [$$n.getField(1), $$n.getField(2)] [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
-                                      -- ASSIGN  |PARTITIONED|
-                                        project ([$$205, $$n]) [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          assign [$$205, $$n] <- [$$212, $$n2] [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
-                                          -- ASSIGN  |PARTITIONED|
-                                            exchange [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              replicate [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
-                                              -- REPLICATE  |PARTITIONED|
-                                                exchange [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  data-scan []<-[$$212, $$n2] <- tpch.nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.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: 1.0, op-cost: 0.0, total-cost: 5.0]
-                          -- HASH_PARTITION_EXCHANGE [$$206]  |PARTITIONED|
-                            project ([$$206]) [cardinality: 1.0, op-cost: 0.0, total-cost: 5.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              assign [$$206] <- [$$213] [cardinality: 1.0, op-cost: 0.0, total-cost: 5.0]
-                              -- ASSIGN  |PARTITIONED|
-                                exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 5.0]
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  replicate [cardinality: 1.0, op-cost: 0.0, total-cost: 5.0]
-                                  -- REPLICATE  |PARTITIONED|
-                                    exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 5.0]
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      project ([$$213]) [cardinality: 1.0, op-cost: 0.0, total-cost: 5.0]
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        select (eq($$r2.getField(1), "EUROPE")) [cardinality: 1.0, op-cost: 0.0, total-cost: 5.0]
-                                        -- STREAM_SELECT  |PARTITIONED|
-                                          exchange [cardinality: 5.0, op-cost: 0.0, total-cost: 5.0]
+                                                                  empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                exchange [cardinality: 1000.0, op-cost: 3948.0, total-cost: 4948.0]
+                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                  assign [$$247, $$246, $$245, $$237, $$239, $$225] <- [$$s.getField(6), $$s.getField(4), $$s.getField(2), $$s.getField(5), $$s.getField(1), $$s.getField(3)] project: [$$237, $$239, $$245, $$246, $$247, $$225, $$201] [cardinality: 1000.0, op-cost: 0.0, total-cost: 1000.0]
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    assign [$$201, $$s] <- [$$211, $$s2] project: [$$201, $$s] [cardinality: 1000.0, op-cost: 0.0, total-cost: 1000.0]
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      exchange [cardinality: 1000.0, op-cost: 3948.0, total-cost: 4948.0]
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        replicate [cardinality: 1000.0, op-cost: 0.0, total-cost: 1000.0]
+                                                        -- REPLICATE  |PARTITIONED|
+                                                          exchange [cardinality: 1000.0, op-cost: 3948.0, total-cost: 4948.0]
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            data-scan []<-[$$211, $$s2] <- tpch.supplier [cardinality: 1000.0, op-cost: 1000.0, total-cost: 1000.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: 25.0, op-cost: 0.0, total-cost: 25.0]
+                                -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
+                                  assign [$$238, $$221] <- [$$n.getField(1), $$n.getField(2)] project: [$$238, $$221, $$205] [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
+                                  -- ASSIGN  |PARTITIONED|
+                                    assign [$$205, $$n] <- [$$212, $$n2] project: [$$205, $$n] [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
+                                    -- ASSIGN  |PARTITIONED|
+                                      exchange [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        replicate [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
+                                        -- REPLICATE  |PARTITIONED|
+                                          exchange [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            data-scan []<-[$$213, $$r2] <- tpch.region [cardinality: 5.0, op-cost: 5.0, total-cost: 5.0]
+                                            data-scan []<-[$$212, $$n2] <- tpch.nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.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: 1.0, op-cost: 0.0, total-cost: 5.0]
+                        -- HASH_PARTITION_EXCHANGE [$$206]  |PARTITIONED|
+                          assign [$$206] <- [$$213] project: [$$206] [cardinality: 1.0, op-cost: 0.0, total-cost: 5.0]
+                          -- ASSIGN  |PARTITIONED|
+                            exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 5.0]
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              replicate [cardinality: 1.0, op-cost: 0.0, total-cost: 5.0]
+                              -- REPLICATE  |PARTITIONED|
+                                exchange [cardinality: 1.0, op-cost: 0.0, total-cost: 5.0]
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  select (eq($$r2.getField(1), "EUROPE")) project: [$$213] [cardinality: 1.0, op-cost: 0.0, total-cost: 5.0]
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    exchange [cardinality: 5.0, op-cost: 0.0, total-cost: 5.0]
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      data-scan []<-[$$213, $$r2] <- tpch.region [cardinality: 5.0, op-cost: 5.0, total-cost: 5.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_cbo/tpch/query-plans/query-plans.05.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.05.plan
index f68dc83..14e91cb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.05.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.05.plan
@@ -4,97 +4,89 @@
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit 10 [cardinality: 30577.54, op-cost: 0.0, total-cost: 1353253.2]
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$122]) [cardinality: 30577.54, op-cost: 0.0, total-cost: 1353253.2]
-      -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$122] <- [{"$1": 3, "l_orderkey": $$l_orderkey, "revenue": $$134, "o_orderdate": $$o_orderdate, "o_shippriority": $$o_shippriority}] [cardinality: 30577.54, op-cost: 0.0, total-cost: 1353253.2]
-        -- ASSIGN  |PARTITIONED|
-          exchange [cardinality: 30577.54, op-cost: 0.0, total-cost: 1353253.2]
-          -- SORT_MERGE_EXCHANGE [$$134(DESC), $$o_orderdate(ASC) ]  |PARTITIONED|
-            limit 10 [cardinality: 30577.54, op-cost: 0.0, total-cost: 1353253.2]
-            -- STREAM_LIMIT  |PARTITIONED|
-              exchange [cardinality: 30577.54, op-cost: 0.0, total-cost: 1353253.2]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (topK: 10) (DESC, $$134) (ASC, $$o_orderdate) [cardinality: 30577.54, op-cost: 0.0, total-cost: 1353253.2]
-                -- STABLE_SORT [topK: 10] [$$134(DESC), $$o_orderdate(ASC)]  |PARTITIONED|
-                  exchange [cardinality: 30577.54, op-cost: 0.0, total-cost: 1353253.2]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    group by ([$$l_orderkey := $$142; $$o_orderdate := $$143; $$o_shippriority := $$144]) decor ([]) {
-                              aggregate [$$134] <- [global-sql-sum-serial($$141)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- AGGREGATE  |LOCAL|
-                                nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                           } [cardinality: 30577.54, op-cost: 30577.54, total-cost: 1353253.2]
-                    -- EXTERNAL_GROUP_BY[$$142, $$143, $$144]  |PARTITIONED|
-                      exchange [cardinality: 30577.54, op-cost: 0.0, total-cost: 1322675.66]
-                      -- HASH_PARTITION_EXCHANGE [$$142, $$143, $$144]  |PARTITIONED|
-                        group by ([$$142 := $$131; $$143 := $$127; $$144 := $$125]) decor ([]) {
-                                  aggregate [$$141] <- [local-sql-sum-serial(numeric-multiply($$139, numeric-subtract(1, $$140)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- AGGREGATE  |LOCAL|
-                                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                               } [cardinality: 30577.54, op-cost: 30577.54, total-cost: 1322675.66]
-                        -- EXTERNAL_GROUP_BY[$$131, $$127, $$125]  |PARTITIONED|
-                          exchange [cardinality: 30577.54, op-cost: 0.0, total-cost: 1292098.12]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            project ([$$139, $$140, $$131, $$127, $$125]) [cardinality: 30577.54, op-cost: 0.0, total-cost: 1292098.12]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              exchange [cardinality: 30577.54, op-cost: 0.0, total-cost: 1292098.12]
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                join (eq($$131, $$129)) [cardinality: 30577.54, op-cost: 369779.81, total-cost: 1292098.12]
-                                -- HYBRID_HASH_JOIN [$$131][$$129]  |PARTITIONED|
-                                  exchange [cardinality: 310738.1, op-cost: 0.0, total-cost: 600572.0]
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    project ([$$139, $$140, $$131]) [cardinality: 310738.1, op-cost: 0.0, total-cost: 600572.0]
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      select (gt($$l.getField(10), "1995-03-22")) [cardinality: 310738.1, op-cost: 0.0, total-cost: 600572.0]
-                                      -- STREAM_SELECT  |PARTITIONED|
-                                        assign [$$140, $$139] <- [$$l.getField(6), $$l.getField(5)] [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
-                                        -- ASSIGN  |PARTITIONED|
-                                          project ([$$131, $$l]) [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            exchange [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
+      assign [$$122] <- [{"$1": 3, "l_orderkey": $$l_orderkey, "revenue": $$134, "o_orderdate": $$o_orderdate, "o_shippriority": $$o_shippriority}] project: [$$122] [cardinality: 30577.54, op-cost: 0.0, total-cost: 1353253.2]
+      -- ASSIGN  |PARTITIONED|
+        exchange [cardinality: 30577.54, op-cost: 0.0, total-cost: 1353253.2]
+        -- SORT_MERGE_EXCHANGE [$$134(DESC), $$o_orderdate(ASC) ]  |PARTITIONED|
+          limit 10 [cardinality: 30577.54, op-cost: 0.0, total-cost: 1353253.2]
+          -- STREAM_LIMIT  |PARTITIONED|
+            exchange [cardinality: 30577.54, op-cost: 0.0, total-cost: 1353253.2]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              order (topK: 10) (DESC, $$134) (ASC, $$o_orderdate) [cardinality: 30577.54, op-cost: 0.0, total-cost: 1353253.2]
+              -- STABLE_SORT [topK: 10] [$$134(DESC), $$o_orderdate(ASC)]  |PARTITIONED|
+                exchange [cardinality: 30577.54, op-cost: 0.0, total-cost: 1353253.2]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  group by ([$$l_orderkey := $$142; $$o_orderdate := $$143; $$o_shippriority := $$144]) decor ([]) {
+                            aggregate [$$134] <- [global-sql-sum-serial($$141)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- AGGREGATE  |LOCAL|
+                              nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                         } [cardinality: 30577.54, op-cost: 30577.54, total-cost: 1353253.2]
+                  -- EXTERNAL_GROUP_BY[$$142, $$143, $$144]  |PARTITIONED|
+                    exchange [cardinality: 30577.54, op-cost: 0.0, total-cost: 1322675.66]
+                    -- HASH_PARTITION_EXCHANGE [$$142, $$143, $$144]  |PARTITIONED|
+                      group by ([$$142 := $$131; $$143 := $$127; $$144 := $$125]) decor ([]) {
+                                aggregate [$$141] <- [local-sql-sum-serial(numeric-multiply($$139, numeric-subtract(1, $$140)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- AGGREGATE  |LOCAL|
+                                  nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                             } [cardinality: 30577.54, op-cost: 30577.54, total-cost: 1322675.66]
+                      -- EXTERNAL_GROUP_BY[$$131, $$127, $$125]  |PARTITIONED|
+                        exchange [cardinality: 30577.54, op-cost: 0.0, total-cost: 1292098.12]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          project ([$$139, $$140, $$131, $$127, $$125]) [cardinality: 30577.54, op-cost: 0.0, total-cost: 1292098.12]
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            exchange [cardinality: 30577.54, op-cost: 0.0, total-cost: 1292098.12]
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              join (eq($$131, $$129)) [cardinality: 30577.54, op-cost: 369779.81, total-cost: 1292098.12]
+                              -- HYBRID_HASH_JOIN [$$131][$$129]  |PARTITIONED|
+                                exchange [cardinality: 310738.1, op-cost: 0.0, total-cost: 600572.0]
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  select (gt($$l.getField(10), "1995-03-22")) project: [$$139, $$140, $$131] [cardinality: 310738.1, op-cost: 0.0, total-cost: 600572.0]
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    assign [$$140, $$139] <- [$$l.getField(6), $$l.getField(5)] [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
+                                    -- ASSIGN  |PARTITIONED|
+                                      project ([$$131, $$l]) [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        exchange [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          data-scan []<-[$$130, $$131, $$l] <- tpch.lineitem [cardinality: 600572.0, op-cost: 600572.0, total-cost: 600572.0]
+                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              data-scan []<-[$$130, $$131, $$l] <- tpch.lineitem [cardinality: 600572.0, op-cost: 600572.0, total-cost: 600572.0]
+                                              empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                exchange [cardinality: 14760.43, op-cost: 59041.71, total-cost: 321746.32]
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  project ([$$127, $$125, $$129]) [cardinality: 14760.43, op-cost: 0.0, total-cost: 262704.61]
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    exchange [cardinality: 14760.43, op-cost: 59041.71, total-cost: 321746.32]
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      join (eq($$128, $$136)) [cardinality: 14760.43, op-cost: 85682.03, total-cost: 262704.61]
+                                      -- HYBRID_HASH_JOIN [$$136][$$128]  |PARTITIONED|
+                                        exchange [cardinality: 73659.45, op-cost: 0.0, total-cost: 150000.0]
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          select (lt($$127, "1995-03-22")) [cardinality: 73659.45, op-cost: 0.0, total-cost: 150000.0]
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            assign [$$125, $$136, $$127] <- [$$o.getField(7), $$o.getField(1), $$o.getField(4)] project: [$$129, $$125, $$136, $$127] [cardinality: 150000.0, op-cost: 0.0, total-cost: 150000.0]
+                                            -- ASSIGN  |PARTITIONED|
+                                              exchange [cardinality: 150000.0, op-cost: 0.0, total-cost: 150000.0]
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                data-scan []<-[$$129, $$o] <- tpch.orders [cardinality: 150000.0, op-cost: 150000.0, total-cost: 150000.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: 3005.64, op-cost: 12022.58, total-cost: 27022.58]
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          select (eq($$c.getField(6), "BUILDING")) project: [$$128] [cardinality: 3005.64, op-cost: 0.0, total-cost: 15000.0]
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            exchange [cardinality: 15000.0, op-cost: 12022.58, total-cost: 27022.58]
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              data-scan []<-[$$128, $$c] <- tpch.customer [cardinality: 15000.0, op-cost: 15000.0, total-cost: 15000.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: 14760.43, op-cost: 59041.71, total-cost: 321746.32]
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                    project ([$$127, $$125, $$129]) [cardinality: 14760.43, op-cost: 0.0, total-cost: 262704.61]
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      exchange [cardinality: 14760.43, op-cost: 59041.71, total-cost: 321746.32]
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        join (eq($$128, $$136)) [cardinality: 14760.43, op-cost: 85682.03, total-cost: 262704.61]
-                                        -- HYBRID_HASH_JOIN [$$136][$$128]  |PARTITIONED|
-                                          exchange [cardinality: 73659.45, op-cost: 0.0, total-cost: 150000.0]
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            select (lt($$127, "1995-03-22")) [cardinality: 73659.45, op-cost: 0.0, total-cost: 150000.0]
-                                            -- STREAM_SELECT  |PARTITIONED|
-                                              project ([$$129, $$125, $$136, $$127]) [cardinality: 150000.0, op-cost: 0.0, total-cost: 150000.0]
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                assign [$$125, $$136, $$127] <- [$$o.getField(7), $$o.getField(1), $$o.getField(4)] [cardinality: 150000.0, op-cost: 0.0, total-cost: 150000.0]
-                                                -- ASSIGN  |PARTITIONED|
-                                                  exchange [cardinality: 150000.0, op-cost: 0.0, total-cost: 150000.0]
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    data-scan []<-[$$129, $$o] <- tpch.orders [cardinality: 150000.0, op-cost: 150000.0, total-cost: 150000.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: 3005.64, op-cost: 12022.58, total-cost: 27022.58]
-                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                            project ([$$128]) [cardinality: 3005.64, op-cost: 0.0, total-cost: 15000.0]
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              select (eq($$c.getField(6), "BUILDING")) [cardinality: 3005.64, op-cost: 0.0, total-cost: 15000.0]
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                exchange [cardinality: 15000.0, op-cost: 12022.58, total-cost: 27022.58]
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  data-scan []<-[$$128, $$c] <- tpch.customer [cardinality: 15000.0, op-cost: 15000.0, total-cost: 15000.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_cbo/tpch/query-plans/query-plans.06.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.06.plan
index 59335a3..e7c52a5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.06.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.06.plan
@@ -2,99 +2,91 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 997.0, op-cost: 0.0, total-cost: 610643.36]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$74]) [cardinality: 997.0, op-cost: 0.0, total-cost: 610643.36]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$74] <- [{"$1": 4, "o_orderpriority": $$o_orderpriority, "order_count": $$81}] [cardinality: 997.0, op-cost: 0.0, total-cost: 610643.36]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 997.0, op-cost: 0.0, total-cost: 610643.36]
-        -- SORT_MERGE_EXCHANGE [$$o_orderpriority(ASC) ]  |PARTITIONED|
-          group by ([$$o_orderpriority := $$91]) decor ([]) {
-                    aggregate [$$81] <- [agg-sql-sum($$90)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 610643.36]
-          -- SORT_GROUP_BY[$$91]  |PARTITIONED|
-            exchange [cardinality: 997.0, op-cost: 0.0, total-cost: 610643.36]
-            -- HASH_PARTITION_EXCHANGE [$$91]  |PARTITIONED|
-              group by ([$$91 := $$75]) decor ([]) {
-                        aggregate [$$90] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 610643.36]
-              -- SORT_GROUP_BY[$$75]  |PARTITIONED|
-                exchange [cardinality: 997.0, op-cost: 0.0, total-cost: 610643.36]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$75]) [cardinality: 997.0, op-cost: 0.0, total-cost: 610643.36]
+    assign [$$74] <- [{"$1": 4, "o_orderpriority": $$o_orderpriority, "order_count": $$81}] project: [$$74] [cardinality: 997.0, op-cost: 0.0, total-cost: 610643.36]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 997.0, op-cost: 0.0, total-cost: 610643.36]
+      -- SORT_MERGE_EXCHANGE [$$o_orderpriority(ASC) ]  |PARTITIONED|
+        group by ([$$o_orderpriority := $$91]) decor ([]) {
+                  aggregate [$$81] <- [agg-sql-sum($$90)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 610643.36]
+        -- SORT_GROUP_BY[$$91]  |PARTITIONED|
+          exchange [cardinality: 997.0, op-cost: 0.0, total-cost: 610643.36]
+          -- HASH_PARTITION_EXCHANGE [$$91]  |PARTITIONED|
+            group by ([$$91 := $$75]) decor ([]) {
+                      aggregate [$$90] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 610643.36]
+            -- SORT_GROUP_BY[$$75]  |PARTITIONED|
+              exchange [cardinality: 997.0, op-cost: 0.0, total-cost: 610643.36]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                select (neq($$80, 0)) project: [$$75] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STREAM_SELECT  |PARTITIONED|
+                  project ([$$80, $$75]) [cardinality: 997.0, op-cost: 0.0, total-cost: 610643.36]
                   -- STREAM_PROJECT  |PARTITIONED|
-                    select (neq($$80, 0)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_SELECT  |PARTITIONED|
-                      project ([$$80, $$75]) [cardinality: 997.0, op-cost: 0.0, total-cost: 610643.36]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        exchange [cardinality: 997.0, op-cost: 0.0, total-cost: 610643.36]
+                    exchange [cardinality: 997.0, op-cost: 0.0, total-cost: 610643.36]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      group by ([$$87 := $$89]) decor ([$$75]) {
+                                aggregate [$$80] <- [sum-serial($$88)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- AGGREGATE  |LOCAL|
+                                  nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                             } [cardinality: 997.0, op-cost: 15928.9, total-cost: 610643.36]
+                      -- EXTERNAL_GROUP_BY[$$89]  |PARTITIONED|
+                        exchange [cardinality: 997.0, op-cost: 0.0, total-cost: 594714.46]
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          group by ([$$87 := $$89]) decor ([$$75]) {
-                                    aggregate [$$80] <- [sum-serial($$88)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          group by ([$$89 := $$77]) decor ([$$75]) {
+                                    aggregate [$$88] <- [agg-count({"l": $$l})] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                     -- AGGREGATE  |LOCAL|
-                                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                 } [cardinality: 997.0, op-cost: 15928.9, total-cost: 610643.36]
-                          -- EXTERNAL_GROUP_BY[$$89]  |PARTITIONED|
-                            exchange [cardinality: 997.0, op-cost: 0.0, total-cost: 594714.46]
+                                      select (not(is-missing($$86))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- STREAM_SELECT  |LOCAL|
+                                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                 } [cardinality: 997.0, op-cost: 222357.23, total-cost: 594714.46]
+                          -- PRE_CLUSTERED_GROUP_BY[$$77]  |PARTITIONED|
+                            exchange [cardinality: 6349.95, op-cost: 0.0, total-cost: 372357.23]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              group by ([$$89 := $$77]) decor ([$$75]) {
-                                        aggregate [$$88] <- [agg-count({"l": $$l})] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- AGGREGATE  |LOCAL|
-                                          select (not(is-missing($$86))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- STREAM_SELECT  |LOCAL|
-                                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                     } [cardinality: 997.0, op-cost: 222357.23, total-cost: 594714.46]
-                              -- PRE_CLUSTERED_GROUP_BY[$$77]  |PARTITIONED|
-                                exchange [cardinality: 6349.95, op-cost: 0.0, total-cost: 372357.23]
+                              order (ASC, $$77) [cardinality: 6349.95, op-cost: 222357.23, total-cost: 372357.23]
+                              -- STABLE_SORT [$$77(ASC)]  |PARTITIONED|
+                                exchange [cardinality: 6349.95, op-cost: 0.0, total-cost: 150000.0]
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  order (ASC, $$77) [cardinality: 6349.95, op-cost: 222357.23, total-cost: 372357.23]
-                                  -- STABLE_SORT [$$77(ASC)]  |PARTITIONED|
+                                  project ([$$75, $$l, $$86, $$77]) [cardinality: 6349.95, op-cost: 0.0, total-cost: 150000.0]
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     exchange [cardinality: 6349.95, op-cost: 0.0, total-cost: 150000.0]
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      project ([$$75, $$l, $$86, $$77]) [cardinality: 6349.95, op-cost: 0.0, total-cost: 150000.0]
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                      left outer join (eq($$79, $$77)) [cardinality: 15928.9, op-cost: 382625.54, total-cost: 1515823.09]
+                                      -- HYBRID_HASH_JOIN [$$77][$$79]  |PARTITIONED|
                                         exchange [cardinality: 6349.95, op-cost: 0.0, total-cost: 150000.0]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          left outer join (eq($$79, $$77)) [cardinality: 15928.9, op-cost: 382625.54, total-cost: 1515823.09]
-                                          -- HYBRID_HASH_JOIN [$$77][$$79]  |PARTITIONED|
-                                            exchange [cardinality: 6349.95, op-cost: 0.0, total-cost: 150000.0]
-                                            -- HASH_PARTITION_EXCHANGE [$$77]  |PARTITIONED|
-                                              project ([$$75, $$77]) [cardinality: 6349.95, op-cost: 0.0, total-cost: 150000.0]
+                                        -- HASH_PARTITION_EXCHANGE [$$77]  |PARTITIONED|
+                                          select (and(ge($$76, "1996-05-01"), lt($$76, "1996-08-01"))) project: [$$75, $$77] [cardinality: 6349.95, op-cost: 0.0, total-cost: 150000.0]
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            assign [$$75, $$76] <- [$$o.getField(5), $$o.getField(4)] project: [$$77, $$75, $$76] [cardinality: 150000.0, op-cost: 0.0, total-cost: 150000.0]
+                                            -- ASSIGN  |PARTITIONED|
+                                              exchange [cardinality: 150000.0, op-cost: 0.0, total-cost: 150000.0]
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                data-scan []<-[$$77, $$o] <- tpch.orders [cardinality: 150000.0, op-cost: 150000.0, total-cost: 150000.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 [$$79]  |PARTITIONED|
+                                          assign [$$86] <- [true] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- ASSIGN  |PARTITIONED|
+                                            select (lt($$l.getField(11), $$l.getField(12))) [cardinality: 376275.59, op-cost: 0.0, total-cost: 600572.0]
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              project ([$$79, $$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                               -- STREAM_PROJECT  |PARTITIONED|
-                                                select (and(ge($$76, "1996-05-01"), lt($$76, "1996-08-01"))) [cardinality: 6349.95, op-cost: 0.0, total-cost: 150000.0]
-                                                -- STREAM_SELECT  |PARTITIONED|
-                                                  project ([$$77, $$75, $$76]) [cardinality: 150000.0, op-cost: 0.0, total-cost: 150000.0]
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    assign [$$75, $$76] <- [$$o.getField(5), $$o.getField(4)] [cardinality: 150000.0, op-cost: 0.0, total-cost: 150000.0]
-                                                    -- ASSIGN  |PARTITIONED|
-                                                      exchange [cardinality: 150000.0, op-cost: 0.0, total-cost: 150000.0]
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        data-scan []<-[$$77, $$o] <- tpch.orders [cardinality: 150000.0, op-cost: 150000.0, total-cost: 150000.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 [$$79]  |PARTITIONED|
-                                              assign [$$86] <- [true] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- ASSIGN  |PARTITIONED|
-                                                select (lt($$l.getField(11), $$l.getField(12))) [cardinality: 376275.59, op-cost: 0.0, total-cost: 600572.0]
-                                                -- STREAM_SELECT  |PARTITIONED|
-                                                  project ([$$79, $$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 []<-[$$78, $$79, $$l] <- tpch.lineitem [cardinality: 600572.0, op-cost: 600572.0, total-cost: 600572.0]
+                                                  -- DATASOURCE_SCAN  |PARTITIONED|
                                                     exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      data-scan []<-[$$78, $$79, $$l] <- tpch.lineitem [cardinality: 600572.0, op-cost: 600572.0, total-cost: 600572.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|
+                                                      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_cbo/tpch/query-plans/query-plans.07.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.07.plan
index 4269fe6..91cea89 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.07.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.07.plan
@@ -1,160 +1,144 @@
-distribute result [$$149] [cardinality: 25.0, op-cost: 0.0, total-cost: 1574886.92]
+distribute result [$$149] [cardinality: 25.0, op-cost: 0.0, total-cost: 1555771.69]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 25.0, op-cost: 0.0, total-cost: 1574886.92]
+  exchange [cardinality: 25.0, op-cost: 0.0, total-cost: 1555771.69]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$149]) [cardinality: 25.0, op-cost: 0.0, total-cost: 1574886.92]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$149] <- [{"$1": 5, "n_name": $$n_name, "revenue": $$165}] [cardinality: 25.0, op-cost: 0.0, total-cost: 1574886.92]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 25.0, op-cost: 0.0, total-cost: 1574886.92]
-        -- SORT_MERGE_EXCHANGE [$$165(DESC) ]  |PARTITIONED|
-          order (DESC, $$165) [cardinality: 25.0, op-cost: 116.1, total-cost: 1574886.92]
-          -- STABLE_SORT [$$165(DESC)]  |PARTITIONED|
-            exchange [cardinality: 25.0, op-cost: 0.0, total-cost: 1574770.82]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              group by ([$$n_name := $$176]) decor ([]) {
-                        aggregate [$$165] <- [global-sql-sum-serial($$175)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 25.0, op-cost: 38494.17, total-cost: 1574770.82]
-              -- EXTERNAL_GROUP_BY[$$176]  |PARTITIONED|
-                exchange [cardinality: 25.0, op-cost: 0.0, total-cost: 1536276.65]
-                -- HASH_PARTITION_EXCHANGE [$$176]  |PARTITIONED|
-                  group by ([$$176 := $$150]) decor ([]) {
-                            aggregate [$$175] <- [local-sql-sum-serial(numeric-multiply($$173, numeric-subtract(1, $$174)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- AGGREGATE  |LOCAL|
-                              nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                         } [cardinality: 25.0, op-cost: 38494.17, total-cost: 1536276.65]
-                  -- EXTERNAL_GROUP_BY[$$150]  |PARTITIONED|
-                    exchange [cardinality: 38494.17, op-cost: 0.0, total-cost: 1497782.48]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      project ([$$173, $$174, $$150]) [cardinality: 38494.17, op-cost: 0.0, total-cost: 1497782.48]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        exchange [cardinality: 38494.17, op-cost: 0.0, total-cost: 1497782.48]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          join (and(eq($$154, $$160), eq($$170, $$159))) [cardinality: 38494.17, op-cost: 23429.97, total-cost: 1497782.48]
-                          -- HYBRID_HASH_JOIN [$$160, $$170][$$154, $$159]  |PARTITIONED|
-                            exchange [cardinality: 19481.97, op-cost: 0.0, total-cost: 1469404.52]
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              project ([$$173, $$174, $$150, $$160, $$170]) [cardinality: 19481.97, op-cost: 0.0, total-cost: 1469404.52]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                exchange [cardinality: 19481.97, op-cost: 0.0, total-cost: 1469404.52]
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  join (eq($$158, $$156)) [cardinality: 19481.97, op-cost: 620582.08, total-cost: 1469404.52]
-                                  -- HYBRID_HASH_JOIN [$$158][$$156]  |PARTITIONED|
-                                    exchange [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      project ([$$173, $$174, $$170, $$158]) [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
+    assign [$$149] <- [{"$1": 5, "n_name": $$n_name, "revenue": $$165}] project: [$$149] [cardinality: 25.0, op-cost: 0.0, total-cost: 1555771.69]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 25.0, op-cost: 0.0, total-cost: 1555771.69]
+      -- SORT_MERGE_EXCHANGE [$$165(DESC) ]  |PARTITIONED|
+        order (DESC, $$165) [cardinality: 25.0, op-cost: 116.1, total-cost: 1555771.69]
+        -- STABLE_SORT [$$165(DESC)]  |PARTITIONED|
+          exchange [cardinality: 25.0, op-cost: 0.0, total-cost: 1555655.59]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            group by ([$$n_name := $$176]) decor ([]) {
+                      aggregate [$$165] <- [global-sql-sum-serial($$175)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 25.0, op-cost: 34190.86, total-cost: 1555655.59]
+            -- EXTERNAL_GROUP_BY[$$176]  |PARTITIONED|
+              exchange [cardinality: 25.0, op-cost: 0.0, total-cost: 1521464.73]
+              -- HASH_PARTITION_EXCHANGE [$$176]  |PARTITIONED|
+                group by ([$$176 := $$150]) decor ([]) {
+                          aggregate [$$175] <- [local-sql-sum-serial(numeric-multiply($$173, numeric-subtract(1, $$174)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- AGGREGATE  |LOCAL|
+                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                       } [cardinality: 25.0, op-cost: 34190.86, total-cost: 1521464.73]
+                -- EXTERNAL_GROUP_BY[$$150]  |PARTITIONED|
+                  exchange [cardinality: 34190.86, op-cost: 0.0, total-cost: 1487273.87]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    project ([$$173, $$174, $$150]) [cardinality: 34190.86, op-cost: 0.0, total-cost: 1487273.87]
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      exchange [cardinality: 34190.86, op-cost: 0.0, total-cost: 1487273.87]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        join (and(eq($$154, $$160), eq($$170, $$159))) [cardinality: 34190.86, op-cost: 21043.43, total-cost: 1487273.87]
+                        -- HYBRID_HASH_JOIN [$$160, $$170][$$154, $$159]  |PARTITIONED|
+                          exchange [cardinality: 17095.43, op-cost: 0.0, total-cost: 1461282.44]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            project ([$$173, $$174, $$150, $$160, $$170]) [cardinality: 17095.43, op-cost: 0.0, total-cost: 1461282.44]
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              exchange [cardinality: 17095.43, op-cost: 0.0, total-cost: 1461282.44]
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                join (eq($$158, $$156)) [cardinality: 17095.43, op-cost: 618200.06, total-cost: 1461282.44]
+                                -- HYBRID_HASH_JOIN [$$158][$$156]  |PARTITIONED|
+                                  exchange [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    assign [$$174, $$173, $$170] <- [$$l.getField(6), $$l.getField(5), $$l.getField(2)] project: [$$173, $$174, $$170, $$158] [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
+                                    -- ASSIGN  |PARTITIONED|
+                                      project ([$$158, $$l]) [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
                                       -- STREAM_PROJECT  |PARTITIONED|
-                                        assign [$$174, $$173, $$170] <- [$$l.getField(6), $$l.getField(5), $$l.getField(2)] [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
-                                        -- ASSIGN  |PARTITIONED|
-                                          project ([$$158, $$l]) [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            exchange [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              data-scan []<-[$$157, $$158, $$l] <- tpch.lineitem [cardinality: 600572.0, op-cost: 600572.0, total-cost: 600572.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: 4861.28, op-cost: 19445.1, total-cost: 248250.43]
-                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                      project ([$$150, $$160, $$156]) [cardinality: 4861.28, op-cost: 0.0, total-cost: 228805.33]
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        exchange [cardinality: 4861.28, op-cost: 19445.1, total-cost: 248250.43]
+                                        exchange [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          join (eq($$155, $$167)) [cardinality: 4861.28, op-cost: 34995.7, total-cost: 228805.33]
-                                          -- HYBRID_HASH_JOIN [$$167][$$155]  |PARTITIONED|
-                                            exchange [cardinality: 21307.62, op-cost: 0.0, total-cost: 150000.0]
+                                          data-scan []<-[$$157, $$158, $$l] <- tpch.lineitem [cardinality: 600572.0, op-cost: 600572.0, total-cost: 600572.0]
+                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              project ([$$156, $$167]) [cardinality: 21307.62, op-cost: 0.0, total-cost: 150000.0]
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                select (and(ge($$151, "1993-01-01"), lt($$151, "1994-01-01"))) [cardinality: 21307.62, op-cost: 0.0, total-cost: 150000.0]
-                                                -- STREAM_SELECT  |PARTITIONED|
-                                                  project ([$$156, $$167, $$151]) [cardinality: 150000.0, op-cost: 0.0, total-cost: 150000.0]
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    assign [$$167, $$151] <- [$$o.getField(1), $$o.getField(4)] [cardinality: 150000.0, op-cost: 0.0, total-cost: 150000.0]
+                                              empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  exchange [cardinality: 4265.77, op-cost: 17063.08, total-cost: 242510.39]
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    project ([$$150, $$160, $$156]) [cardinality: 4265.77, op-cost: 0.0, total-cost: 225447.31]
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      exchange [cardinality: 4265.77, op-cost: 17063.08, total-cost: 242510.39]
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        join (eq($$155, $$167)) [cardinality: 4265.77, op-cost: 33318.91, total-cost: 225447.31]
+                                        -- HYBRID_HASH_JOIN [$$167][$$155]  |PARTITIONED|
+                                          exchange [cardinality: 21307.62, op-cost: 0.0, total-cost: 150000.0]
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            select (and(ge($$151, "1993-01-01"), lt($$151, "1994-01-01"))) project: [$$156, $$167] [cardinality: 21307.62, op-cost: 0.0, total-cost: 150000.0]
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              assign [$$167, $$151] <- [$$o.getField(1), $$o.getField(4)] project: [$$156, $$167, $$151] [cardinality: 150000.0, op-cost: 0.0, total-cost: 150000.0]
+                                              -- ASSIGN  |PARTITIONED|
+                                                exchange [cardinality: 150000.0, op-cost: 0.0, total-cost: 150000.0]
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  data-scan []<-[$$156, $$o] <- tpch.orders [cardinality: 150000.0, op-cost: 150000.0, total-cost: 150000.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: 3002.82, op-cost: 12011.29, total-cost: 42128.4]
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            project ([$$150, $$160, $$155]) [cardinality: 3002.82, op-cost: 0.0, total-cost: 30117.11]
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              exchange [cardinality: 3002.82, op-cost: 12011.29, total-cost: 42128.4]
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                join (eq($$153, $$160)) [cardinality: 3002.82, op-cost: 15034.11, total-cost: 30117.11]
+                                                -- HYBRID_HASH_JOIN [$$153][$$160]  |PARTITIONED|
+                                                  exchange [cardinality: 15000.0, op-cost: 0.0, total-cost: 15000.0]
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    assign [$$153] <- [$$c.getField(3)] project: [$$155, $$153] [cardinality: 15000.0, op-cost: 0.0, total-cost: 15000.0]
                                                     -- ASSIGN  |PARTITIONED|
-                                                      exchange [cardinality: 150000.0, op-cost: 0.0, total-cost: 150000.0]
+                                                      exchange [cardinality: 15000.0, op-cost: 0.0, total-cost: 15000.0]
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        data-scan []<-[$$156, $$o] <- tpch.orders [cardinality: 150000.0, op-cost: 150000.0, total-cost: 150000.0]
+                                                        data-scan []<-[$$155, $$c] <- tpch.customer [cardinality: 15000.0, op-cost: 15000.0, total-cost: 15000.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: 3422.02, op-cost: 13688.08, total-cost: 43809.64]
-                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                              project ([$$150, $$160, $$155]) [cardinality: 3422.02, op-cost: 0.0, total-cost: 30121.56]
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                exchange [cardinality: 3422.02, op-cost: 13688.08, total-cost: 43809.64]
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  join (eq($$153, $$160)) [cardinality: 3422.02, op-cost: 15036.33, total-cost: 30121.56]
-                                                  -- HYBRID_HASH_JOIN [$$153][$$160]  |PARTITIONED|
-                                                    exchange [cardinality: 15000.0, op-cost: 0.0, total-cost: 15000.0]
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      project ([$$155, $$153]) [cardinality: 15000.0, op-cost: 0.0, total-cost: 15000.0]
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        assign [$$153] <- [$$c.getField(3)] [cardinality: 15000.0, op-cost: 0.0, total-cost: 15000.0]
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          exchange [cardinality: 15000.0, op-cost: 0.0, total-cost: 15000.0]
+                                                  exchange [cardinality: 5.0, op-cost: 20.0, total-cost: 83.0]
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    project ([$$150, $$160]) [cardinality: 5.0, op-cost: 0.0, total-cost: 63.0]
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      exchange [cardinality: 5.0, op-cost: 20.0, total-cost: 83.0]
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        join (eq($$163, $$161)) [cardinality: 5.0, op-cost: 29.0, total-cost: 63.0]
+                                                        -- HYBRID_HASH_JOIN [$$163][$$161]  |PARTITIONED|
+                                                          exchange [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            data-scan []<-[$$155, $$c] <- tpch.customer [cardinality: 15000.0, op-cost: 15000.0, total-cost: 15000.0]
-                                                            -- DATASOURCE_SCAN  |PARTITIONED|
-                                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                            assign [$$150, $$163] <- [$$n.getField(1), $$n.getField(2)] project: [$$150, $$160, $$163] [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              exchange [cardinality: 25.0, op-cost: 0.0, total-cost: 25.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: 5.56, op-cost: 22.22, total-cost: 85.22]
-                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                      project ([$$150, $$160]) [cardinality: 5.56, op-cost: 0.0, total-cost: 63.0]
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        exchange [cardinality: 5.56, op-cost: 22.22, total-cost: 85.22]
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          join (eq($$163, $$161)) [cardinality: 5.56, op-cost: 29.0, total-cost: 63.0]
-                                                          -- HYBRID_HASH_JOIN [$$163][$$161]  |PARTITIONED|
-                                                            exchange [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              project ([$$150, $$160, $$163]) [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                assign [$$150, $$163] <- [$$n.getField(1), $$n.getField(2)] [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  exchange [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
+                                                                data-scan []<-[$$160, $$n] <- tpch.nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.0]
+                                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    data-scan []<-[$$160, $$n] <- tpch.nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.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: 1.0, op-cost: 4.0, total-cost: 9.0]
-                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                              project ([$$161]) [cardinality: 1.0, op-cost: 0.0, total-cost: 5.0]
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                select (eq($$r.getField(1), "AFRICA")) [cardinality: 1.0, op-cost: 0.0, total-cost: 5.0]
-                                                                -- STREAM_SELECT  |PARTITIONED|
-                                                                  exchange [cardinality: 5.0, op-cost: 4.0, total-cost: 9.0]
+                                                                    empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          exchange [cardinality: 1.0, op-cost: 4.0, total-cost: 9.0]
+                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                            select (eq($$r.getField(1), "AFRICA")) project: [$$161] [cardinality: 1.0, op-cost: 0.0, total-cost: 5.0]
+                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                              exchange [cardinality: 5.0, op-cost: 4.0, total-cost: 9.0]
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                data-scan []<-[$$161, $$r] <- tpch.region [cardinality: 5.0, op-cost: 5.0, total-cost: 5.0]
+                                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    data-scan []<-[$$161, $$r] <- tpch.region [cardinality: 5.0, op-cost: 5.0, total-cost: 5.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: 1000.0, op-cost: 3948.0, total-cost: 4948.0]
-                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                              project ([$$154, $$159]) [cardinality: 1000.0, op-cost: 0.0, total-cost: 1000.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                assign [$$154] <- [$$s.getField(3)] [cardinality: 1000.0, op-cost: 0.0, total-cost: 1000.0]
-                                -- ASSIGN  |PARTITIONED|
-                                  exchange [cardinality: 1000.0, op-cost: 3948.0, total-cost: 4948.0]
+                                                                    empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          exchange [cardinality: 1000.0, op-cost: 3948.0, total-cost: 4948.0]
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            assign [$$154] <- [$$s.getField(3)] project: [$$154, $$159] [cardinality: 1000.0, op-cost: 0.0, total-cost: 1000.0]
+                            -- ASSIGN  |PARTITIONED|
+                              exchange [cardinality: 1000.0, op-cost: 3948.0, total-cost: 4948.0]
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                data-scan []<-[$$159, $$s] <- tpch.supplier [cardinality: 1000.0, op-cost: 1000.0, total-cost: 1000.0]
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    data-scan []<-[$$159, $$s] <- tpch.supplier [cardinality: 1000.0, op-cost: 1000.0, total-cost: 1000.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|
+                                    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_cbo/tpch/query-plans/query-plans.08.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.08.plan
index c11d3d2..1a2110b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.08.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.08.plan
@@ -1,166 +1,148 @@
-distribute result [$$186] [cardinality: 2.52, op-cost: 0.0, total-cost: 991063.1]
+distribute result [$$186] [cardinality: 2.52, op-cost: 0.0, total-cost: 988897.43]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 2.52, op-cost: 0.0, total-cost: 991063.1]
+  exchange [cardinality: 2.52, op-cost: 0.0, total-cost: 988897.43]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$186]) [cardinality: 2.52, op-cost: 0.0, total-cost: 991063.1]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$186] <- [{"$1": 7, "supp_nation": $$supp_nation, "cust_nation": $$cust_nation, "l_year": $$l_year, "revenue": $$200}] [cardinality: 2.52, op-cost: 0.0, total-cost: 991063.1]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 2.52, op-cost: 0.0, total-cost: 991063.1]
-        -- SORT_MERGE_EXCHANGE [$$supp_nation(ASC), $$cust_nation(ASC), $$l_year(ASC) ]  |PARTITIONED|
-          order (ASC, $$supp_nation) (ASC, $$cust_nation) (ASC, $$l_year) [cardinality: 2.52, op-cost: 3.36, total-cost: 991063.1]
-          -- STABLE_SORT [$$supp_nation(ASC), $$cust_nation(ASC), $$l_year(ASC)]  |PARTITIONED|
-            exchange [cardinality: 2.52, op-cost: 0.0, total-cost: 991059.74]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              group by ([$$supp_nation := $$217; $$cust_nation := $$218; $$l_year := $$219]) decor ([]) {
-                        aggregate [$$200] <- [global-sql-sum-serial($$216)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 2.52, op-cost: 600.05, total-cost: 991059.74]
-              -- EXTERNAL_GROUP_BY[$$217, $$218, $$219]  |PARTITIONED|
-                exchange [cardinality: 2.52, op-cost: 0.0, total-cost: 990459.69]
-                -- HASH_PARTITION_EXCHANGE [$$217, $$218, $$219]  |PARTITIONED|
-                  group by ([$$217 := $$191; $$218 := $$192; $$219 := $$214]) decor ([]) {
-                            aggregate [$$216] <- [local-sql-sum-serial($$184)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- AGGREGATE  |LOCAL|
-                              nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                         } [cardinality: 2.52, op-cost: 600.05, total-cost: 990459.69]
-                  -- EXTERNAL_GROUP_BY[$$191, $$192, $$214]  |PARTITIONED|
-                    exchange [cardinality: 600.05, op-cost: 0.0, total-cost: 989859.64]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      project ([$$184, $$191, $$192, $$214]) [cardinality: 600.05, op-cost: 0.0, total-cost: 989859.64]
+    assign [$$186] <- [{"$1": 7, "supp_nation": $$supp_nation, "cust_nation": $$cust_nation, "l_year": $$l_year, "revenue": $$200}] project: [$$186] [cardinality: 2.52, op-cost: 0.0, total-cost: 988897.43]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 2.52, op-cost: 0.0, total-cost: 988897.43]
+      -- SORT_MERGE_EXCHANGE [$$supp_nation(ASC), $$cust_nation(ASC), $$l_year(ASC) ]  |PARTITIONED|
+        order (ASC, $$supp_nation) (ASC, $$cust_nation) (ASC, $$l_year) [cardinality: 2.52, op-cost: 3.36, total-cost: 988897.43]
+        -- STABLE_SORT [$$supp_nation(ASC), $$cust_nation(ASC), $$l_year(ASC)]  |PARTITIONED|
+          exchange [cardinality: 2.52, op-cost: 0.0, total-cost: 988894.07]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            group by ([$$supp_nation := $$217; $$cust_nation := $$218; $$l_year := $$219]) decor ([]) {
+                      aggregate [$$200] <- [global-sql-sum-serial($$216)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 2.52, op-cost: 577.39, total-cost: 988894.07]
+            -- EXTERNAL_GROUP_BY[$$217, $$218, $$219]  |PARTITIONED|
+              exchange [cardinality: 2.52, op-cost: 0.0, total-cost: 988316.68]
+              -- HASH_PARTITION_EXCHANGE [$$217, $$218, $$219]  |PARTITIONED|
+                group by ([$$217 := $$191; $$218 := $$192; $$219 := $$214]) decor ([]) {
+                          aggregate [$$216] <- [local-sql-sum-serial($$184)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- AGGREGATE  |LOCAL|
+                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                       } [cardinality: 2.52, op-cost: 577.39, total-cost: 988316.68]
+                -- EXTERNAL_GROUP_BY[$$191, $$192, $$214]  |PARTITIONED|
+                  exchange [cardinality: 577.39, op-cost: 0.0, total-cost: 987739.29]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    assign [$$184, $$214] <- [numeric-multiply($$211, numeric-subtract(1, $$212)), get-year(date($$190))] project: [$$184, $$191, $$192, $$214] [cardinality: 577.39, op-cost: 0.0, total-cost: 987739.29]
+                    -- ASSIGN  |PARTITIONED|
+                      project ([$$191, $$192, $$211, $$212, $$190]) [cardinality: 577.39, op-cost: 0.0, total-cost: 987739.29]
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$184, $$214] <- [numeric-multiply($$211, numeric-subtract(1, $$212)), get-year(date($$190))] [cardinality: 600.05, op-cost: 0.0, total-cost: 989859.64]
-                        -- ASSIGN  |PARTITIONED|
-                          project ([$$191, $$192, $$211, $$212, $$190]) [cardinality: 600.05, op-cost: 0.0, total-cost: 989859.64]
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            exchange [cardinality: 600.05, op-cost: 0.0, total-cost: 989859.64]
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              join (and(eq($$201, $$199), eq($$197, $$210))) [cardinality: 600.05, op-cost: 29625.83, total-cost: 989859.64]
-                              -- HYBRID_HASH_JOIN [$$199, $$210][$$201, $$197]  |PARTITIONED|
-                                exchange [cardinality: 150141.11, op-cost: 14611.72, total-cost: 164611.72]
-                                -- HASH_PARTITION_EXCHANGE [$$199, $$210]  |PARTITIONED|
-                                  project ([$$191, $$192, $$211, $$212, $$190, $$199, $$210]) [cardinality: 150141.11, op-cost: 0.0, total-cost: 150000.0]
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    assign [$$210] <- [$$o.getField(1)] [cardinality: 150141.11, op-cost: 0.0, total-cost: 150000.0]
-                                    -- ASSIGN  |PARTITIONED|
-                                      project ([$$212, $$211, $$190, $$191, $$192, $$199, $$o]) [cardinality: 150141.11, op-cost: 0.0, total-cost: 150000.0]
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        exchange [cardinality: 150141.11, op-cost: 14611.72, total-cost: 164611.72]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          unnest-map [$$196, $$o] <- index-search("orders", 0, "Default", "tpch", "orders", true, true, 1, $$195, 1, $$195, true, true, true) [cardinality: 150141.11, op-cost: 150000.0, total-cost: 150000.0]
-                                          -- BTREE_SEARCH  |PARTITIONED|
-                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              order (ASC, $$195) [cardinality: 150141.11, op-cost: 150000.0, total-cost: 150000.0]
-                                              -- STABLE_SORT [$$195(ASC)]  |PARTITIONED|
-                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- HASH_PARTITION_EXCHANGE [$$195]  |PARTITIONED|
-                                                  project ([$$195, $$212, $$211, $$190, $$191, $$192, $$199]) [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|
-                                                      join (eq($$193, $$204)) [cardinality: 14597.97, op-cost: 180551.78, total-cost: 784226.26]
-                                                      -- HYBRID_HASH_JOIN [$$204][$$193]  |PARTITIONED|
-                                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          select (and(ge($$190, "1995-01-01"), le($$190, "1996-12-31"))) [cardinality: 180228.1, op-cost: 0.0, total-cost: 600572.0]
-                                                          -- STREAM_SELECT  |PARTITIONED|
-                                                            project ([$$195, $$212, $$211, $$204, $$190]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              assign [$$212, $$211, $$204, $$190] <- [$$l.getField(6), $$l.getField(5), $$l.getField(2), $$l.getField(10)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                project ([$$195, $$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                -- STREAM_PROJECT  |PARTITIONED|
+                        exchange [cardinality: 577.39, op-cost: 0.0, total-cost: 987739.29]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          join (and(eq($$201, $$199), eq($$197, $$210))) [cardinality: 577.39, op-cost: 29434.39, total-cost: 987739.29]
+                          -- HYBRID_HASH_JOIN [$$199, $$210][$$201, $$197]  |PARTITIONED|
+                            exchange [cardinality: 150141.11, op-cost: 14420.28, total-cost: 164420.28]
+                            -- HASH_PARTITION_EXCHANGE [$$199, $$210]  |PARTITIONED|
+                              assign [$$210] <- [$$o.getField(1)] project: [$$191, $$192, $$211, $$212, $$190, $$199, $$210] [cardinality: 150141.11, op-cost: 0.0, total-cost: 150000.0]
+                              -- ASSIGN  |PARTITIONED|
+                                project ([$$212, $$211, $$190, $$191, $$192, $$199, $$o]) [cardinality: 150141.11, op-cost: 0.0, total-cost: 150000.0]
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  exchange [cardinality: 150141.11, op-cost: 14420.28, total-cost: 164420.28]
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    unnest-map [$$196, $$o] <- index-search("orders", 0, "Default", "tpch", "orders", true, true, 1, $$195, 1, $$195, true, true, true) [cardinality: 150141.11, op-cost: 150000.0, total-cost: 150000.0]
+                                    -- BTREE_SEARCH  |PARTITIONED|
+                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        order (ASC, $$195) [cardinality: 150141.11, op-cost: 150000.0, total-cost: 150000.0]
+                                        -- STABLE_SORT [$$195(ASC)]  |PARTITIONED|
+                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- HASH_PARTITION_EXCHANGE [$$195]  |PARTITIONED|
+                                            project ([$$195, $$212, $$211, $$190, $$191, $$192, $$199]) [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|
+                                                join (eq($$193, $$204)) [cardinality: 14406.71, op-cost: 180543.69, total-cost: 784210.07]
+                                                -- HYBRID_HASH_JOIN [$$204][$$193]  |PARTITIONED|
+                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    select (and(ge($$190, "1995-01-01"), le($$190, "1996-12-31"))) [cardinality: 180228.1, op-cost: 0.0, total-cost: 600572.0]
+                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                      assign [$$212, $$211, $$204, $$190] <- [$$l.getField(6), $$l.getField(5), $$l.getField(2), $$l.getField(10)] project: [$$195, $$212, $$211, $$204, $$190] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        project ([$$195, $$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 []<-[$$194, $$195, $$l] <- tpch.lineitem [cardinality: 600572.0, op-cost: 600572.0, total-cost: 600572.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]
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    project ([$$191, $$192, $$199, $$193]) [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|
+                                                        join (eq($$205, $$198)) [cardinality: 78.9, op-cost: 995.4, total-cost: 2778.8]
+                                                        -- HYBRID_HASH_JOIN [$$205][$$198]  |PARTITIONED|
+                                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            assign [$$205] <- [$$s.getField(3)] project: [$$193, $$205] [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]
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                data-scan []<-[$$193, $$s] <- tpch.supplier [cardinality: 1000.0, op-cost: 1000.0, total-cost: 1000.0]
+                                                                -- DATASOURCE_SCAN  |PARTITIONED|
                                                                   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    data-scan []<-[$$194, $$195, $$l] <- tpch.lineitem [cardinality: 600572.0, op-cost: 600572.0, total-cost: 600572.0]
-                                                                    -- DATASOURCE_SCAN  |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]
+                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                            join (or(and(eq($$191, "KENYA"), eq($$192, "PERU")), and(eq($$191, "PERU"), eq($$192, "KENYA")))) [cardinality: 2.1, op-cost: 625.0, total-cost: 775.0]
+                                                            -- NESTED_LOOP  |PARTITIONED|
+                                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                assign [$$191, $$198] <- [$$192, $$199] project: [$$191, $$198] [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]
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                    -- REPLICATE  |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]
-                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                          project ([$$191, $$192, $$199, $$193]) [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|
-                                                              join (eq($$205, $$198)) [cardinality: 80.92, op-cost: 995.4, total-cost: 2778.8]
-                                                              -- HYBRID_HASH_JOIN [$$205][$$198]  |PARTITIONED|
-                                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  project ([$$193, $$205]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    assign [$$205] <- [$$s.getField(3)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                        assign [$$192] <- [$$n2.getField(1)] project: [$$192, $$199] [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]
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            data-scan []<-[$$199, $$n2] <- tpch.nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.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]
+                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                -- REPLICATE  |PARTITIONED|
+                                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    assign [$$192] <- [$$n2.getField(1)] project: [$$192, $$199] [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]
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        data-scan []<-[$$193, $$s] <- tpch.supplier [cardinality: 1000.0, op-cost: 1000.0, total-cost: 1000.0]
+                                                                        data-scan []<-[$$199, $$n2] <- tpch.nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.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]
-                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                  join (or(and(eq($$191, "KENYA"), eq($$192, "PERU")), and(eq($$191, "PERU"), eq($$192, "KENYA")))) [cardinality: 2.1, op-cost: 625.0, total-cost: 775.0]
-                                                                  -- NESTED_LOOP  |PARTITIONED|
-                                                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      project ([$$191, $$198]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        assign [$$191, $$198] <- [$$192, $$199] [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]
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                            -- REPLICATE  |PARTITIONED|
-                                                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                project ([$$192, $$199]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                  assign [$$192] <- [$$n2.getField(1)] [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]
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      data-scan []<-[$$199, $$n2] <- tpch.nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.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]
-                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                      replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                      -- REPLICATE  |PARTITIONED|
-                                                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          project ([$$192, $$199]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            assign [$$192] <- [$$n2.getField(1)] [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]
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                data-scan []<-[$$199, $$n2] <- tpch.nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.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: 15000.0, op-cost: 15014.11, total-cost: 30014.11]
+                            -- HASH_PARTITION_EXCHANGE [$$201, $$197]  |PARTITIONED|
+                              assign [$$201] <- [$$c.getField(3)] project: [$$201, $$197] [cardinality: 15000.0, op-cost: 0.0, total-cost: 15000.0]
+                              -- ASSIGN  |PARTITIONED|
                                 exchange [cardinality: 15000.0, op-cost: 15014.11, total-cost: 30014.11]
-                                -- HASH_PARTITION_EXCHANGE [$$201, $$197]  |PARTITIONED|
-                                  project ([$$201, $$197]) [cardinality: 15000.0, op-cost: 0.0, total-cost: 15000.0]
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    assign [$$201] <- [$$c.getField(3)] [cardinality: 15000.0, op-cost: 0.0, total-cost: 15000.0]
-                                    -- ASSIGN  |PARTITIONED|
-                                      exchange [cardinality: 15000.0, op-cost: 15014.11, total-cost: 30014.11]
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        data-scan []<-[$$197, $$c] <- tpch.customer [cardinality: 15000.0, op-cost: 15000.0, total-cost: 15000.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|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  data-scan []<-[$$197, $$c] <- tpch.customer [cardinality: 15000.0, op-cost: 15000.0, total-cost: 15000.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_cbo/tpch/query-plans/query-plans.09.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.09.plan
index 1e2dd48..a22503e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.09.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.09.plan
@@ -1,192 +1,170 @@
-distribute result [$$199] [cardinality: 2.0, op-cost: 0.0, total-cost: 1259243.16]
+distribute result [$$199] [cardinality: 2.0, op-cost: 0.0, total-cost: 1258961.42]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 1259243.16]
+  exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 1258961.42]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$199]) [cardinality: 2.0, op-cost: 0.0, total-cost: 1259243.16]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$199] <- [{"$1": 8, "o_year": $$o_year, "mkt_share": numeric-divide($$214, $$215)}] [cardinality: 2.0, op-cost: 0.0, total-cost: 1259243.16]
-      -- ASSIGN  |PARTITIONED|
-        exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 1259243.16]
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          group by ([$$o_year := $$238]) decor ([]) {
-                    aggregate [$$214, $$215] <- [global-sql-sum-serial($$236), global-sql-sum-serial($$237)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 2.0, op-cost: 193.7, total-cost: 1259243.16]
-          -- EXTERNAL_GROUP_BY[$$238]  |PARTITIONED|
-            exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 1259049.46]
-            -- HASH_PARTITION_EXCHANGE [$$238]  |PARTITIONED|
-              group by ([$$238 := $$200]) decor ([]) {
-                        aggregate [$$236, $$237] <- [local-sql-sum-serial(switch-case(true, eq($$232, "PERU"), numeric-multiply($$230, numeric-subtract(1, $$231)), 0)), local-sql-sum-serial(numeric-multiply($$230, numeric-subtract(1, $$231)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 2.0, op-cost: 193.7, total-cost: 1259049.46]
-              -- EXTERNAL_GROUP_BY[$$200]  |PARTITIONED|
-                exchange [cardinality: 193.7, op-cost: 0.0, total-cost: 1258855.76]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$232, $$230, $$231, $$200]) [cardinality: 193.7, op-cost: 0.0, total-cost: 1258855.76]
+    assign [$$199] <- [{"$1": 8, "o_year": $$o_year, "mkt_share": numeric-divide($$214, $$215)}] project: [$$199] [cardinality: 2.0, op-cost: 0.0, total-cost: 1258961.42]
+    -- ASSIGN  |PARTITIONED|
+      exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 1258961.42]
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        group by ([$$o_year := $$238]) decor ([]) {
+                  aggregate [$$214, $$215] <- [global-sql-sum-serial($$236), global-sql-sum-serial($$237)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 2.0, op-cost: 167.75, total-cost: 1258961.42]
+        -- EXTERNAL_GROUP_BY[$$238]  |PARTITIONED|
+          exchange [cardinality: 2.0, op-cost: 0.0, total-cost: 1258793.67]
+          -- HASH_PARTITION_EXCHANGE [$$238]  |PARTITIONED|
+            group by ([$$238 := $$200]) decor ([]) {
+                      aggregate [$$236, $$237] <- [local-sql-sum-serial(switch-case(true, eq($$232, "PERU"), numeric-multiply($$230, numeric-subtract(1, $$231)), 0)), local-sql-sum-serial(numeric-multiply($$230, numeric-subtract(1, $$231)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 2.0, op-cost: 167.75, total-cost: 1258793.67]
+            -- EXTERNAL_GROUP_BY[$$200]  |PARTITIONED|
+              exchange [cardinality: 167.75, op-cost: 0.0, total-cost: 1258625.92]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                assign [$$200] <- [get-year(date($$201))] project: [$$232, $$230, $$231, $$200] [cardinality: 167.75, op-cost: 0.0, total-cost: 1258625.92]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$230, $$231, $$201, $$232]) [cardinality: 167.75, op-cost: 0.0, total-cost: 1258625.92]
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$200] <- [get-year(date($$201))] [cardinality: 193.7, op-cost: 0.0, total-cost: 1258855.76]
-                    -- ASSIGN  |PARTITIONED|
-                      project ([$$230, $$231, $$201, $$232]) [cardinality: 193.7, op-cost: 0.0, total-cost: 1258855.76]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        exchange [cardinality: 193.7, op-cost: 0.0, total-cost: 1258855.76]
+                    exchange [cardinality: 167.75, op-cost: 0.0, total-cost: 1258625.92]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      join (eq($$216, $$209)) [cardinality: 167.75, op-cost: 267.75, total-cost: 1258625.92]
+                      -- HYBRID_HASH_JOIN [$$216][$$209]  |PARTITIONED|
+                        exchange [cardinality: 987.0, op-cost: 0.0, total-cost: 1000.0]
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          join (eq($$216, $$209)) [cardinality: 193.7, op-cost: 288.86, total-cost: 1258855.76]
-                          -- HYBRID_HASH_JOIN [$$216][$$209]  |PARTITIONED|
-                            exchange [cardinality: 987.0, op-cost: 0.0, total-cost: 1000.0]
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              project ([$$230, $$231, $$201, $$216]) [cardinality: 987.0, op-cost: 0.0, total-cost: 1000.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                assign [$$216] <- [$$s.getField(3)] [cardinality: 987.0, op-cost: 0.0, total-cost: 1000.0]
-                                -- ASSIGN  |PARTITIONED|
-                                  project ([$$230, $$231, $$201, $$s]) [cardinality: 987.0, op-cost: 0.0, total-cost: 1000.0]
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    exchange [cardinality: 987.0, op-cost: 0.0, total-cost: 1000.0]
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      unnest-map [$$203, $$s] <- index-search("supplier", 0, "Default", "tpch", "supplier", true, true, 1, $$223, 1, $$223, true, true, true) [cardinality: 987.0, op-cost: 1000.0, total-cost: 1000.0]
-                                      -- BTREE_SEARCH  |PARTITIONED|
-                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          order (ASC, $$223) [cardinality: 987.0, op-cost: 1000.0, total-cost: 1000.0]
-                                          -- STABLE_SORT [$$223(ASC)]  |PARTITIONED|
-                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- HASH_PARTITION_EXCHANGE [$$223]  |PARTITIONED|
-                                              project ([$$230, $$231, $$201, $$223]) [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|
-                                                  join (and(eq($$212, $$210), eq($$218, $$208))) [cardinality: 191.16, op-cost: 938.74, total-cost: 1256721.24]
-                                                  -- HYBRID_HASH_JOIN [$$210, $$218][$$212, $$208]  |PARTITIONED|
+                          assign [$$216] <- [$$s.getField(3)] project: [$$230, $$231, $$201, $$216] [cardinality: 987.0, op-cost: 0.0, total-cost: 1000.0]
+                          -- ASSIGN  |PARTITIONED|
+                            project ([$$230, $$231, $$201, $$s]) [cardinality: 987.0, op-cost: 0.0, total-cost: 1000.0]
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              exchange [cardinality: 987.0, op-cost: 0.0, total-cost: 1000.0]
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                unnest-map [$$203, $$s] <- index-search("supplier", 0, "Default", "tpch", "supplier", true, true, 1, $$223, 1, $$223, true, true, true) [cardinality: 987.0, op-cost: 1000.0, total-cost: 1000.0]
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    order (ASC, $$223) [cardinality: 987.0, op-cost: 1000.0, total-cost: 1000.0]
+                                    -- STABLE_SORT [$$223(ASC)]  |PARTITIONED|
+                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- HASH_PARTITION_EXCHANGE [$$223]  |PARTITIONED|
+                                        project ([$$230, $$231, $$201, $$223]) [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|
+                                            join (and(eq($$212, $$210), eq($$218, $$208))) [cardinality: 167.75, op-cost: 938.74, total-cost: 1256721.24]
+                                            -- HYBRID_HASH_JOIN [$$210, $$218][$$212, $$208]  |PARTITIONED|
+                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                assign [$$218] <- [$$c.getField(3)] project: [$$230, $$231, $$201, $$223, $$210, $$218] [cardinality: 838.74, op-cost: 4189.55, total-cost: 1255657.5]
+                                                -- ASSIGN  |PARTITIONED|
+                                                  project ([$$230, $$231, $$223, $$210, $$201, $$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|
-                                                      project ([$$230, $$231, $$201, $$223, $$210, $$218]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        assign [$$218] <- [$$c.getField(3)] [cardinality: 838.74, op-cost: 4189.55, total-cost: 1255657.5]
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          project ([$$230, $$231, $$223, $$210, $$201, $$c]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                      unnest-map [$$207, $$c] <- index-search("customer", 0, "Default", "tpch", "customer", true, true, 1, $$226, 1, $$226, true, true, true) [cardinality: 15014.11, op-cost: 15000.0, total-cost: 15000.0]
+                                                      -- BTREE_SEARCH  |PARTITIONED|
+                                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          order (ASC, $$226) [cardinality: 15014.11, op-cost: 15000.0, total-cost: 15000.0]
+                                                          -- STABLE_SORT [$$226(ASC)]  |PARTITIONED|
                                                             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              unnest-map [$$207, $$c] <- index-search("customer", 0, "Default", "tpch", "customer", true, true, 1, $$226, 1, $$226, true, true, true) [cardinality: 15014.11, op-cost: 15000.0, total-cost: 15000.0]
-                                                              -- BTREE_SEARCH  |PARTITIONED|
-                                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  order (ASC, $$226) [cardinality: 15014.11, op-cost: 15000.0, total-cost: 15000.0]
-                                                                  -- STABLE_SORT [$$226(ASC)]  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$226]  |PARTITIONED|
+                                                              select (and(ge($$201, "1995-01-01"), le($$201, "1996-12-31"))) [cardinality: 837.9, op-cost: 14137.91, total-cost: 1248116.35]
+                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                assign [$$226, $$201] <- [$$o.getField(1), $$o.getField(4)] project: [$$230, $$231, $$223, $$210, $$226, $$201] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  project ([$$230, $$231, $$223, $$210, $$o]) [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]
-                                                                    -- HASH_PARTITION_EXCHANGE [$$226]  |PARTITIONED|
-                                                                      select (and(ge($$201, "1995-01-01"), le($$201, "1996-12-31"))) [cardinality: 837.9, op-cost: 14137.91, total-cost: 1248116.35]
-                                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                                        project ([$$230, $$231, $$223, $$210, $$226, $$201]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          assign [$$226, $$201] <- [$$o.getField(1), $$o.getField(4)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            project ([$$230, $$231, $$223, $$210, $$o]) [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 [$$206, $$o] <- index-search("orders", 0, "Default", "tpch", "orders", true, true, 1, $$205, 1, $$205, true, true, true) [cardinality: 44449.67, op-cost: 150000.0, total-cost: 150000.0]
-                                                                                -- BTREE_SEARCH  |PARTITIONED|
-                                                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                    order (ASC, $$205) [cardinality: 44449.67, op-cost: 150000.0, total-cost: 150000.0]
-                                                                                    -- STABLE_SORT [$$205(ASC)]  |PARTITIONED|
-                                                                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                      -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
-                                                                                        project ([$$230, $$231, $$223, $$205, $$210]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      unnest-map [$$206, $$o] <- index-search("orders", 0, "Default", "tpch", "orders", true, true, 1, $$205, 1, $$205, true, true, true) [cardinality: 44449.67, op-cost: 150000.0, total-cost: 150000.0]
+                                                                      -- BTREE_SEARCH  |PARTITIONED|
+                                                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          order (ASC, $$205) [cardinality: 44449.67, op-cost: 150000.0, total-cost: 150000.0]
+                                                                          -- STABLE_SORT [$$205(ASC)]  |PARTITIONED|
+                                                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                            -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
+                                                                              project ([$$230, $$231, $$223, $$205, $$210]) [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|
+                                                                                  join (eq($$202, $$221)) [cardinality: 2827.58, op-cost: 601513.27, total-cost: 1222668.12]
+                                                                                  -- HYBRID_HASH_JOIN [$$221][$$202]  |PARTITIONED|
+                                                                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      assign [$$231, $$230, $$223, $$221] <- [$$l.getField(6), $$l.getField(5), $$l.getField(2), $$l.getField(1)] project: [$$230, $$231, $$223, $$205, $$221] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        project ([$$205, $$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|
-                                                                                            join (eq($$202, $$221)) [cardinality: 2827.58, op-cost: 601513.27, total-cost: 1222668.12]
-                                                                                            -- HYBRID_HASH_JOIN [$$221][$$202]  |PARTITIONED|
+                                                                                            data-scan []<-[$$204, $$205, $$l] <- tpch.lineitem [cardinality: 600572.0, op-cost: 600572.0, total-cost: 600572.0]
+                                                                                            -- DATASOURCE_SCAN  |PARTITIONED|
                                                                                               exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                project ([$$230, $$231, $$223, $$205, $$221]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                  assign [$$231, $$230, $$223, $$221] <- [$$l.getField(6), $$l.getField(5), $$l.getField(2), $$l.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                    project ([$$205, $$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 []<-[$$204, $$205, $$l] <- tpch.lineitem [cardinality: 600572.0, op-cost: 600572.0, total-cost: 600572.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]
-                                                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                join (true) [cardinality: 94.07, op-cost: 197.55, total-cost: 20206.55]
-                                                                                                -- NESTED_LOOP  |PARTITIONED|
-                                                                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                    project ([$$202]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                      select (eq($$p.getField(4), "ECONOMY BURNISHED NICKEL")) [cardinality: 94.07, op-cost: 0.0, total-cost: 20000.0]
-                                                                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                                                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          data-scan []<-[$$202, $$p] <- tpch.part [cardinality: 20000.0, op-cost: 20000.0, total-cost: 20000.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]
-                                                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                    project ([$$210]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                      select (eq($$r.getField(1), "AMERICA")) [cardinality: 1.0, op-cost: 0.0, total-cost: 5.0]
-                                                                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                                                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          data-scan []<-[$$210, $$r] <- tpch.region [cardinality: 5.0, op-cost: 5.0, total-cost: 5.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]
-                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                      project ([$$212, $$208]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        assign [$$212] <- [$$n1.getField(2)] [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|
+                                                                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                      join (true) [cardinality: 94.07, op-cost: 197.55, total-cost: 20206.55]
+                                                                                      -- NESTED_LOOP  |PARTITIONED|
+                                                                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          select (eq($$p.getField(4), "ECONOMY BURNISHED NICKEL")) project: [$$202] [cardinality: 94.07, op-cost: 0.0, total-cost: 20000.0]
+                                                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              data-scan []<-[$$202, $$p] <- tpch.part [cardinality: 20000.0, op-cost: 20000.0, total-cost: 20000.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]
+                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                          select (eq($$r.getField(1), "AMERICA")) project: [$$210] [cardinality: 1.0, op-cost: 0.0, total-cost: 5.0]
+                                                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              data-scan []<-[$$210, $$r] <- tpch.region [cardinality: 5.0, op-cost: 5.0, total-cost: 5.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]
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                assign [$$212] <- [$$n1.getField(2)] project: [$$212, $$208] [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]
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    replicate [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
+                                                    -- REPLICATE  |PARTITIONED|
+                                                      exchange [cardinality: 25.0, op-cost: 100.0, total-cost: 125.0]
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        data-scan []<-[$$208, $$n1] <- tpch.nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.0]
+                                                        -- DATASOURCE_SCAN  |PARTITIONED|
                                                           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            replicate [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
-                                                            -- REPLICATE  |PARTITIONED|
-                                                              exchange [cardinality: 25.0, op-cost: 100.0, total-cost: 125.0]
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                data-scan []<-[$$208, $$n1] <- tpch.nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.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: 25.0, op-cost: 100.0, total-cost: 125.0]
-                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                              project ([$$232, $$209]) [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                assign [$$232] <- [$$n2.getField(1)] [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
-                                -- ASSIGN  |PARTITIONED|
-                                  project ([$$209, $$n2]) [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    assign [$$209, $$n2] <- [$$208, $$n1] [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
-                                    -- ASSIGN  |PARTITIONED|
-                                      exchange [cardinality: 25.0, op-cost: 100.0, total-cost: 125.0]
+                                                            empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        exchange [cardinality: 25.0, op-cost: 100.0, total-cost: 125.0]
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          assign [$$232] <- [$$n2.getField(1)] project: [$$232, $$209] [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
+                          -- ASSIGN  |PARTITIONED|
+                            assign [$$209, $$n2] <- [$$208, $$n1] project: [$$209, $$n2] [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
+                            -- ASSIGN  |PARTITIONED|
+                              exchange [cardinality: 25.0, op-cost: 100.0, total-cost: 125.0]
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                replicate [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
+                                -- REPLICATE  |PARTITIONED|
+                                  exchange [cardinality: 25.0, op-cost: 100.0, total-cost: 125.0]
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    data-scan []<-[$$208, $$n1] <- tpch.nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.0]
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        replicate [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
-                                        -- REPLICATE  |PARTITIONED|
-                                          exchange [cardinality: 25.0, op-cost: 100.0, total-cost: 125.0]
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            data-scan []<-[$$208, $$n1] <- tpch.nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.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|
+                                        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_cbo/tpch/query-plans/query-plans.10.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.10.plan
index 680b321..ca1e816 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.10.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.10.plan
@@ -1,122 +1,114 @@
-distribute result [$$122] [cardinality: 26982.83, op-cost: 0.0, total-cost: 1673953.69]
+distribute result [$$122] [cardinality: 26629.31, op-cost: 0.0, total-cost: 1670441.71]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
-  exchange [cardinality: 26982.83, op-cost: 0.0, total-cost: 1673953.69]
+  exchange [cardinality: 26629.31, op-cost: 0.0, total-cost: 1670441.71]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$122]) [cardinality: 26982.83, op-cost: 0.0, total-cost: 1673953.69]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$122] <- [{"$1": 9, "$2": $$140}] [cardinality: 26982.83, op-cost: 0.0, total-cost: 1673953.69]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$140] <- [agg-sql-sum($$144)] [cardinality: 26982.83, op-cost: 0.0, total-cost: 1673953.69]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 26982.83, op-cost: 0.0, total-cost: 1673953.69]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$144] <- [agg-sql-count(1)] [cardinality: 26982.83, op-cost: 0.0, total-cost: 1673953.69]
-            -- AGGREGATE  |PARTITIONED|
-              exchange [cardinality: 26982.83, op-cost: 0.0, total-cost: 1673953.69]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                unnest-map [$$136, $$o] <- index-search("orders", 0, "Default", "tpch", "orders", true, true, 1, $$132, 1, $$132, true, true, true) [cardinality: 26982.83, op-cost: 134787.25, total-cost: 1673953.69]
-                -- BTREE_SEARCH  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    order (ASC, $$132) [cardinality: 26982.83, op-cost: 134787.25, total-cost: 1673953.69]
-                    -- STABLE_SORT [$$132(ASC)]  |PARTITIONED|
-                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- HASH_PARTITION_EXCHANGE [$$132]  |PARTITIONED|
-                        project ([$$132]) [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|
-                            join (and(eq($$135, $$128), eq($$134, $$125))) [cardinality: 26957.44, op-cost: 615487.09, total-cost: 1431336.67]
-                            -- HYBRID_HASH_JOIN [$$128, $$125][$$135, $$134]  |PARTITIONED|
-                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                project ([$$132, $$128, $$125]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  assign [$$128, $$125] <- [$$l.getField(2), $$l.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ASSIGN  |PARTITIONED|
-                                    project ([$$132, $$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 []<-[$$131, $$132, $$l] <- tpch.lineitem [cardinality: 600572.0, op-cost: 600572.0, total-cost: 600572.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]
-                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                project ([$$134, $$135]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$122] <- [{"$1": 9, "$2": $$140}] project: [$$122] [cardinality: 26629.31, op-cost: 0.0, total-cost: 1670441.71]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$140] <- [agg-sql-sum($$144)] [cardinality: 26629.31, op-cost: 0.0, total-cost: 1670441.71]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 26629.31, op-cost: 0.0, total-cost: 1670441.71]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$144] <- [agg-sql-count(1)] [cardinality: 26629.31, op-cost: 0.0, total-cost: 1670441.71]
+          -- AGGREGATE  |PARTITIONED|
+            exchange [cardinality: 26629.31, op-cost: 0.0, total-cost: 1670441.71]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              unnest-map [$$136, $$o] <- index-search("orders", 0, "Default", "tpch", "orders", true, true, 1, $$132, 1, $$132, true, true, true) [cardinality: 26629.31, op-cost: 133021.33, total-cost: 1670441.71]
+              -- BTREE_SEARCH  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$132) [cardinality: 26629.31, op-cost: 133021.33, total-cost: 1670441.71]
+                  -- STABLE_SORT [$$132(ASC)]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$132]  |PARTITIONED|
+                      project ([$$132]) [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|
+                          join (and(eq($$135, $$128), eq($$134, $$125))) [cardinality: 26604.26, op-cost: 615299.08, total-cost: 1431003.34]
+                          -- HYBRID_HASH_JOIN [$$128, $$125][$$135, $$134]  |PARTITIONED|
+                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              assign [$$128, $$125] <- [$$l.getField(2), $$l.getField(1)] project: [$$132, $$128, $$125] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- ASSIGN  |PARTITIONED|
+                                project ([$$132, $$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|
-                                    join (eq($$138, $$137)) [cardinality: 3587.53, op-cost: 3597.84, total-cost: 200927.47]
-                                    -- HYBRID_HASH_JOIN [$$138][$$137]  |PARTITIONED|
+                                    data-scan []<-[$$131, $$132, $$l] <- tpch.lineitem [cardinality: 600572.0, op-cost: 600572.0, total-cost: 600572.0]
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
                                       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        project ([$$134, $$135, $$138]) [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|
-                                            join (eq($$130, $$135)) [cardinality: 3497.84, op-cost: 4527.53, total-cost: 197204.63]
-                                            -- HYBRID_HASH_JOIN [$$135][$$130]  |PARTITIONED|
-                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
-                                                project ([$$134, $$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|
-                                                    join (eq($$129, $$134)) [cardinality: 3540.53, op-cost: 83612.42, total-cost: 187149.58]
-                                                    -- HYBRID_HASH_JOIN [$$134][$$129]  |PARTITIONED|
-                                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        project ([$$134, $$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 []<-[$$133, $$134, $$135, $$ps] <- tpch.partsupp [cardinality: 80000.0, op-cost: 80000.0, total-cost: 80000.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]
-                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                        project ([$$129]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          select (like($$p.getField(1), "%green%")) [cardinality: 884.29, op-cost: 0.0, total-cost: 20000.0]
-                                                          -- STREAM_SELECT  |PARTITIONED|
-                                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              data-scan []<-[$$129, $$p] <- tpch.part [cardinality: 20000.0, op-cost: 20000.0, total-cost: 20000.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 [$$130]  |PARTITIONED|
-                                                project ([$$130, $$138]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  assign [$$138] <- [$$s.getField(3)] [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|
+                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                              project ([$$134, $$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|
+                                  join (eq($$138, $$137)) [cardinality: 3540.53, op-cost: 3640.53, total-cost: 200970.15]
+                                  -- HYBRID_HASH_JOIN [$$138][$$137]  |PARTITIONED|
+                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      project ([$$134, $$135, $$138]) [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|
+                                          join (eq($$130, $$135)) [cardinality: 3540.53, op-cost: 4527.53, total-cost: 197204.63]
+                                          -- HYBRID_HASH_JOIN [$$135][$$130]  |PARTITIONED|
+                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                            -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
+                                              project ([$$134, $$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|
+                                                  join (eq($$129, $$134)) [cardinality: 3540.53, op-cost: 83612.42, total-cost: 187149.58]
+                                                  -- HYBRID_HASH_JOIN [$$134][$$129]  |PARTITIONED|
                                                     exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      data-scan []<-[$$130, $$s] <- tpch.supplier [cardinality: 1000.0, op-cost: 1000.0, total-cost: 1000.0]
-                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      project ([$$134, $$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|
-                                                          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]
-                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                        project ([$$137]) [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 []<-[$$137, $$n] <- tpch.nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.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|
+                                                          data-scan []<-[$$133, $$134, $$135, $$ps] <- tpch.partsupp [cardinality: 80000.0, op-cost: 80000.0, total-cost: 80000.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]
+                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                      select (like($$p.getField(1), "%green%")) project: [$$129] [cardinality: 884.29, op-cost: 0.0, total-cost: 20000.0]
+                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          data-scan []<-[$$129, $$p] <- tpch.part [cardinality: 20000.0, op-cost: 20000.0, total-cost: 20000.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 [$$130]  |PARTITIONED|
+                                              assign [$$138] <- [$$s.getField(3)] project: [$$130, $$138] [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]
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  data-scan []<-[$$130, $$s] <- tpch.supplier [cardinality: 1000.0, op-cost: 1000.0, total-cost: 1000.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]
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      project ([$$137]) [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 []<-[$$137, $$n] <- tpch.nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.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_cbo/tpch/query-plans/query-plans.11.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.11.plan
index e1947ee..ca45dac 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.11.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/tpch/query-plans/query-plans.11.plan
@@ -1,118 +1,106 @@
-distribute result [$$185] [cardinality: 4550.31, op-cost: 0.0, total-cost: 990713.46]
+distribute result [$$185] [cardinality: 4436.55, op-cost: 0.0, total-cost: 990428.16]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
-  exchange [cardinality: 4550.31, op-cost: 0.0, total-cost: 990713.46]
+  exchange [cardinality: 4436.55, op-cost: 0.0, total-cost: 990428.16]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    limit 20 [cardinality: 4550.31, op-cost: 0.0, total-cost: 990713.46]
+    limit 20 [cardinality: 4436.55, op-cost: 0.0, total-cost: 990428.16]
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$185]) [cardinality: 4550.31, op-cost: 0.0, total-cost: 990713.46]
-      -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$185] <- [{"$1": 10, "c_custkey": $$c_custkey, "c_name": $$c_name, "revenue": $$203, "c_acctbal": $$c_acctbal, "n_name": $$n_name, "c_address": $$c_address, "c_phone": $$c_phone, "c_comment": $$c_comment}] [cardinality: 4550.31, op-cost: 0.0, total-cost: 990713.46]
-        -- ASSIGN  |PARTITIONED|
-          exchange [cardinality: 4550.31, op-cost: 0.0, total-cost: 990713.46]
-          -- SORT_MERGE_EXCHANGE [$$203(DESC) ]  |PARTITIONED|
-            limit 20 [cardinality: 4550.31, op-cost: 0.0, total-cost: 990713.46]
-            -- STREAM_LIMIT  |PARTITIONED|
-              exchange [cardinality: 4550.31, op-cost: 0.0, total-cost: 990713.46]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (topK: 20) (DESC, $$203) [cardinality: 4550.31, op-cost: 0.0, total-cost: 990713.46]
-                -- STABLE_SORT [topK: 20] [$$203(DESC)]  |PARTITIONED|
-                  exchange [cardinality: 4550.31, op-cost: 0.0, total-cost: 990713.46]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    group by ([$$c_custkey := $$211; $$c_name := $$212; $$c_acctbal := $$213; $$c_phone := $$214; $$n_name := $$215; $$c_address := $$216; $$c_comment := $$217]) decor ([]) {
-                              aggregate [$$203] <- [global-sql-sum-serial($$210)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- AGGREGATE  |LOCAL|
-                                nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                           } [cardinality: 4550.31, op-cost: 4550.31, total-cost: 990713.46]
-                    -- EXTERNAL_GROUP_BY[$$211, $$212, $$213, $$214, $$215, $$216, $$217]  |PARTITIONED|
-                      exchange [cardinality: 4550.31, op-cost: 0.0, total-cost: 986163.15]
-                      -- HASH_PARTITION_EXCHANGE [$$211, $$212, $$213, $$214, $$215, $$216, $$217]  |PARTITIONED|
-                        group by ([$$211 := $$195; $$212 := $$187; $$213 := $$188; $$214 := $$189; $$215 := $$190; $$216 := $$191; $$217 := $$192]) decor ([]) {
-                                  aggregate [$$210] <- [local-sql-sum-serial(numeric-multiply($$208, numeric-subtract(1, $$209)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- AGGREGATE  |LOCAL|
-                                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                               } [cardinality: 4550.31, op-cost: 4550.31, total-cost: 986163.15]
-                        -- EXTERNAL_GROUP_BY[$$195, $$187, $$188, $$189, $$190, $$191, $$192]  |PARTITIONED|
-                          exchange [cardinality: 4550.31, op-cost: 0.0, total-cost: 981612.84]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            project ([$$208, $$209, $$195, $$187, $$188, $$189, $$190, $$191, $$192]) [cardinality: 4550.31, op-cost: 0.0, total-cost: 981612.84]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              exchange [cardinality: 4550.31, op-cost: 0.0, total-cost: 981612.84]
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                join (eq($$201, $$199)) [cardinality: 4550.31, op-cost: 4536.55, total-cost: 981612.84]
-                                -- HYBRID_HASH_JOIN [$$201][$$199]  |PARTITIONED|
-                                  exchange [cardinality: 4436.55, op-cost: 0.0, total-cost: 976951.29]
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    project ([$$208, $$209, $$195, $$187, $$188, $$189, $$191, $$192, $$201]) [cardinality: 4436.55, op-cost: 0.0, total-cost: 976951.29]
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      exchange [cardinality: 4436.55, op-cost: 0.0, total-cost: 976951.29]
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        join (eq($$198, $$196)) [cardinality: 4436.55, op-cost: 169494.26, total-cost: 976951.29]
-                                        -- HYBRID_HASH_JOIN [$$198][$$196]  |PARTITIONED|
-                                          exchange [cardinality: 151979.18, op-cost: 0.0, total-cost: 600572.0]
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            project ([$$208, $$209, $$198]) [cardinality: 151979.18, op-cost: 0.0, total-cost: 600572.0]
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              select (eq($$l.getField(8), "R")) [cardinality: 151979.18, op-cost: 0.0, total-cost: 600572.0]
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                assign [$$209, $$208] <- [$$l.getField(6), $$l.getField(5)] [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
-                                                -- ASSIGN  |PARTITIONED|
-                                                  project ([$$198, $$l]) [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    exchange [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      data-scan []<-[$$197, $$198, $$l] <- tpch.lineitem [cardinality: 600572.0, op-cost: 600572.0, total-cost: 600572.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: 15014.11, op-cost: 17515.08, total-cost: 32515.08]
-                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                            project ([$$195, $$187, $$188, $$189, $$191, $$192, $$201, $$196]) [cardinality: 15014.11, op-cost: 0.0, total-cost: 15000.0]
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              assign [$$192, $$191, $$189, $$188, $$187, $$201] <- [$$c.getField(7), $$c.getField(2), $$c.getField(4), $$c.getField(5), $$c.getField(1), $$c.getField(3)] [cardinality: 15014.11, op-cost: 0.0, total-cost: 15000.0]
-                                              -- ASSIGN  |PARTITIONED|
-                                                project ([$$196, $$195, $$c]) [cardinality: 15014.11, op-cost: 0.0, total-cost: 15000.0]
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  exchange [cardinality: 15014.11, op-cost: 17515.08, total-cost: 32515.08]
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    unnest-map [$$195, $$c] <- index-search("customer", 0, "Default", "tpch", "customer", true, true, 1, $$206, 1, $$206, true, true, true) [cardinality: 15014.11, op-cost: 15000.0, total-cost: 15000.0]
-                                                    -- BTREE_SEARCH  |PARTITIONED|
-                                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        order (ASC, $$206) [cardinality: 15014.11, op-cost: 15000.0, total-cost: 15000.0]
-                                                        -- STABLE_SORT [$$206(ASC)]  |PARTITIONED|
-                                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                          -- HASH_PARTITION_EXCHANGE [$$206]  |PARTITIONED|
-                                                            project ([$$196, $$206]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              select (and(lt($$193, "1993-10-01"), ge($$193, "1993-07-01"))) [cardinality: 4374.41, op-cost: 0.0, total-cost: 150000.0]
-                                                              -- STREAM_SELECT  |PARTITIONED|
-                                                                project ([$$196, $$206, $$193]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  assign [$$206, $$193] <- [$$o.getField(1), $$o.getField(4)] [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]
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      data-scan []<-[$$196, $$o] <- tpch.orders [cardinality: 150000.0, op-cost: 150000.0, total-cost: 150000.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: 25.0, op-cost: 100.0, total-cost: 125.0]
-                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                    project ([$$190, $$199]) [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      assign [$$190] <- [$$n.getField(1)] [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
-                                      -- ASSIGN  |PARTITIONED|
-                                        exchange [cardinality: 25.0, op-cost: 100.0, total-cost: 125.0]
+      assign [$$185] <- [{"$1": 10, "c_custkey": $$c_custkey, "c_name": $$c_name, "revenue": $$203, "c_acctbal": $$c_acctbal, "n_name": $$n_name, "c_address": $$c_address, "c_phone": $$c_phone, "c_comment": $$c_comment}] project: [$$185] [cardinality: 4436.55, op-cost: 0.0, total-cost: 990428.16]
+      -- ASSIGN  |PARTITIONED|
+        exchange [cardinality: 4436.55, op-cost: 0.0, total-cost: 990428.16]
+        -- SORT_MERGE_EXCHANGE [$$203(DESC) ]  |PARTITIONED|
+          limit 20 [cardinality: 4436.55, op-cost: 0.0, total-cost: 990428.16]
+          -- STREAM_LIMIT  |PARTITIONED|
+            exchange [cardinality: 4436.55, op-cost: 0.0, total-cost: 990428.16]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              order (topK: 20) (DESC, $$203) [cardinality: 4436.55, op-cost: 0.0, total-cost: 990428.16]
+              -- STABLE_SORT [topK: 20] [$$203(DESC)]  |PARTITIONED|
+                exchange [cardinality: 4436.55, op-cost: 0.0, total-cost: 990428.16]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  group by ([$$c_custkey := $$211; $$c_name := $$212; $$c_acctbal := $$213; $$c_phone := $$214; $$n_name := $$215; $$c_address := $$216; $$c_comment := $$217]) decor ([]) {
+                            aggregate [$$203] <- [global-sql-sum-serial($$210)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- AGGREGATE  |LOCAL|
+                              nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                         } [cardinality: 4436.55, op-cost: 4436.55, total-cost: 990428.16]
+                  -- EXTERNAL_GROUP_BY[$$211, $$212, $$213, $$214, $$215, $$216, $$217]  |PARTITIONED|
+                    exchange [cardinality: 4436.55, op-cost: 0.0, total-cost: 985991.61]
+                    -- HASH_PARTITION_EXCHANGE [$$211, $$212, $$213, $$214, $$215, $$216, $$217]  |PARTITIONED|
+                      group by ([$$211 := $$195; $$212 := $$187; $$213 := $$188; $$214 := $$189; $$215 := $$190; $$216 := $$191; $$217 := $$192]) decor ([]) {
+                                aggregate [$$210] <- [local-sql-sum-serial(numeric-multiply($$208, numeric-subtract(1, $$209)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- AGGREGATE  |LOCAL|
+                                  nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                             } [cardinality: 4436.55, op-cost: 4436.55, total-cost: 985991.61]
+                      -- EXTERNAL_GROUP_BY[$$195, $$187, $$188, $$189, $$190, $$191, $$192]  |PARTITIONED|
+                        exchange [cardinality: 4436.55, op-cost: 0.0, total-cost: 981555.06]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          project ([$$208, $$209, $$195, $$187, $$188, $$189, $$190, $$191, $$192]) [cardinality: 4436.55, op-cost: 0.0, total-cost: 981555.06]
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            exchange [cardinality: 4436.55, op-cost: 0.0, total-cost: 981555.06]
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              join (eq($$198, $$196)) [cardinality: 4436.55, op-cost: 169494.26, total-cost: 981555.06]
+                              -- HYBRID_HASH_JOIN [$$198][$$196]  |PARTITIONED|
+                                exchange [cardinality: 151979.18, op-cost: 0.0, total-cost: 600572.0]
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  select (eq($$l.getField(8), "R")) project: [$$208, $$209, $$198] [cardinality: 151979.18, op-cost: 0.0, total-cost: 600572.0]
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    assign [$$209, $$208] <- [$$l.getField(6), $$l.getField(5)] [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
+                                    -- ASSIGN  |PARTITIONED|
+                                      project ([$$198, $$l]) [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        exchange [cardinality: 600572.0, op-cost: 0.0, total-cost: 600572.0]
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          data-scan []<-[$$199, $$n] <- tpch.nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.0]
+                                          data-scan []<-[$$197, $$198, $$l] <- tpch.lineitem [cardinality: 600572.0, op-cost: 600572.0, total-cost: 600572.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: 4378.77, op-cost: 17515.08, total-cost: 211488.8]
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  project ([$$195, $$187, $$188, $$189, $$190, $$191, $$192, $$196]) [cardinality: 4378.77, op-cost: 0.0, total-cost: 193973.72]
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    exchange [cardinality: 4378.77, op-cost: 17515.08, total-cost: 211488.8]
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      join (eq($$201, $$199)) [cardinality: 4378.77, op-cost: 4478.77, total-cost: 193973.72]
+                                      -- HYBRID_HASH_JOIN [$$201][$$199]  |PARTITIONED|
+                                        exchange [cardinality: 15014.11, op-cost: 0.0, total-cost: 15000.0]
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          assign [$$192, $$191, $$189, $$188, $$187, $$201] <- [$$c.getField(7), $$c.getField(2), $$c.getField(4), $$c.getField(5), $$c.getField(1), $$c.getField(3)] project: [$$195, $$187, $$188, $$189, $$191, $$192, $$196, $$201] [cardinality: 15014.11, op-cost: 0.0, total-cost: 15000.0]
+                                          -- ASSIGN  |PARTITIONED|
+                                            project ([$$196, $$195, $$c]) [cardinality: 15014.11, op-cost: 0.0, total-cost: 15000.0]
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              exchange [cardinality: 15014.11, op-cost: 0.0, total-cost: 15000.0]
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                unnest-map [$$195, $$c] <- index-search("customer", 0, "Default", "tpch", "customer", true, true, 1, $$206, 1, $$206, true, true, true) [cardinality: 15014.11, op-cost: 15000.0, total-cost: 15000.0]
+                                                -- BTREE_SEARCH  |PARTITIONED|
+                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    order (ASC, $$206) [cardinality: 15014.11, op-cost: 15000.0, total-cost: 15000.0]
+                                                    -- STABLE_SORT [$$206(ASC)]  |PARTITIONED|
+                                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                      -- HASH_PARTITION_EXCHANGE [$$206]  |PARTITIONED|
+                                                        select (and(lt($$193, "1993-10-01"), ge($$193, "1993-07-01"))) project: [$$196, $$206] [cardinality: 4374.41, op-cost: 0.0, total-cost: 150000.0]
+                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                          assign [$$206, $$193] <- [$$o.getField(1), $$o.getField(4)] project: [$$196, $$206, $$193] [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]
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              data-scan []<-[$$196, $$o] <- tpch.orders [cardinality: 150000.0, op-cost: 150000.0, total-cost: 150000.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: 25.0, op-cost: 100.0, total-cost: 125.0]
+                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                          assign [$$190] <- [$$n.getField(1)] project: [$$190, $$199] [cardinality: 25.0, op-cost: 0.0, total-cost: 25.0]
+                                          -- ASSIGN  |PARTITIONED|
+                                            exchange [cardinality: 25.0, op-cost: 100.0, total-cost: 125.0]
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              data-scan []<-[$$199, $$n] <- tpch.nation [cardinality: 25.0, op-cost: 25.0, total-cost: 25.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_cbo/view/view-pushdown/view-pushdown.04.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/view/view-pushdown/view-pushdown.04.plan
index 0f6bbef..a5b3307 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/view/view-pushdown/view-pushdown.04.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/view/view-pushdown/view-pushdown.04.plan
@@ -2,33 +2,29 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$94]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$94] <- [{"id": $$109, "review": $$114}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-          order (ASC, $$109) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$109(ASC)]  |UNPARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-              limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_LIMIT  |UNPARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                  project ([$$114, $$109]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$109] <- [int64-default-null($$d.getField("id"))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$94] <- [{"id": $$109, "review": $$114}] project: [$$94] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+        order (ASC, $$109) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$109(ASC)]  |UNPARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+            limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_LIMIT  |UNPARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                assign [$$109] <- [int64-default-null($$d.getField("id"))] project: [$$114, $$109] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_LIMIT  |PARTITIONED|
+                    assign [$$114] <- [string-default-null($$d.getField("review"))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                     -- ASSIGN  |PARTITIONED|
-                      limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_LIMIT  |PARTITIONED|
-                        assign [$$114] <- [string-default-null($$d.getField("review"))] [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]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        data-scan []<-[$$d] <- test.ExternalDataset condition (and(not(is-unknown(int64-default-null($$d.getField("year")))), not(is-unknown(int64-default-null($$d.getField("quarter")))), eq(string-default-null($$d.getField("review")), "good"))) limit 3 [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|
-                            data-scan []<-[$$d] <- test.ExternalDataset condition (and(not(is-unknown(int64-default-null($$d.getField("year")))), not(is-unknown(int64-default-null($$d.getField("quarter")))), eq(string-default-null($$d.getField("review")), "good"))) limit 3 [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|
\ No newline at end of file
+                            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_cbo/view/view-pushdown/view-pushdown.06.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/view/view-pushdown/view-pushdown.06.plan
index 1d3ef0b..98cefbf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/view/view-pushdown/view-pushdown.06.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/view/view-pushdown/view-pushdown.06.plan
@@ -6,17 +6,15 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        project ([$$69]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- STREAM_PROJECT  |PARTITIONED|
-          assign [$$69] <- [{"id": int64-default-null($$d.getField("id")), "review": string-default-null($$d.getField("review"))}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- ASSIGN  |PARTITIONED|
-            limit 3 [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]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                data-scan []<-[$$d] <- test.ExternalDataset limit 3 [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|
+        assign [$$69] <- [{"id": int64-default-null($$d.getField("id")), "review": string-default-null($$d.getField("review"))}] project: [$$69] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- ASSIGN  |PARTITIONED|
+          limit 3 [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]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$d] <- test.ExternalDataset limit 3 [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_cbo/view/view-pushdown/view-pushdown.08.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/view/view-pushdown/view-pushdown.08.plan
index de730dc..f51f439 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/view/view-pushdown/view-pushdown.08.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/view/view-pushdown/view-pushdown.08.plan
@@ -2,31 +2,27 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 35.51]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$88]) [cardinality: 6.0, op-cost: 0.0, total-cost: 35.51]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$88] <- [{"id": $$91, "review": $$95}] [cardinality: 6.0, op-cost: 0.0, total-cost: 35.51]
-      -- ASSIGN  |UNPARTITIONED|
-        exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 35.51]
-        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-          order (ASC, $$91) [cardinality: 6.0, op-cost: 15.51, total-cost: 35.51]
-          -- STABLE_SORT [$$91(ASC)]  |UNPARTITIONED|
-            exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
-            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-              limit 3 [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
-              -- STREAM_LIMIT  |UNPARTITIONED|
-                exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
-                -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                  limit 3 [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
-                  -- STREAM_LIMIT  |PARTITIONED|
-                    project ([$$91, $$95]) [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$95] <- [$$d.getField(1)] [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
-                      -- ASSIGN  |PARTITIONED|
-                        exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
+    assign [$$88] <- [{"id": $$91, "review": $$95}] project: [$$88] [cardinality: 6.0, op-cost: 0.0, total-cost: 35.51]
+    -- ASSIGN  |UNPARTITIONED|
+      exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 35.51]
+      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+        order (ASC, $$91) [cardinality: 6.0, op-cost: 15.51, total-cost: 35.51]
+        -- STABLE_SORT [$$91(ASC)]  |UNPARTITIONED|
+          exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
+          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+            limit 3 [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
+            -- STREAM_LIMIT  |UNPARTITIONED|
+              exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
+              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                limit 3 [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
+                -- STREAM_LIMIT  |PARTITIONED|
+                  assign [$$95] <- [$$d.getField(1)] project: [$$91, $$95] [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
+                  -- ASSIGN  |PARTITIONED|
+                    exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
+                    -- 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: 6.0, op-cost: 20.0, total-cost: 20.0]
+                      -- DATASOURCE_SCAN  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- 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: 6.0, op-cost: 20.0, total-cost: 20.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|
+                          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_cbo/view/view-pushdown/view-pushdown.10.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/view/view-pushdown/view-pushdown.10.plan
index af0c764..ac1a8b6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/view/view-pushdown/view-pushdown.10.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/view/view-pushdown/view-pushdown.10.plan
@@ -4,29 +4,25 @@
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit 3 [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$63]) [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
-      -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$63] <- [{"id": $$65, "review": $$69}] [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
-        -- ASSIGN  |PARTITIONED|
-          exchange [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
-          -- SORT_MERGE_EXCHANGE [$$65(ASC) ]  |PARTITIONED|
-            limit 3 [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
-            -- STREAM_LIMIT  |PARTITIONED|
-              exchange [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (topK: 3) (ASC, $$65) [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
-                -- STABLE_SORT [topK: 3] [$$65(ASC)]  |PARTITIONED|
-                  exchange [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$65, $$69]) [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$69] <- [$$d.getField(1)] [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
-                      -- ASSIGN  |PARTITIONED|
-                        exchange [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
+      assign [$$63] <- [{"id": $$65, "review": $$69}] project: [$$63] [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
+      -- ASSIGN  |PARTITIONED|
+        exchange [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
+        -- SORT_MERGE_EXCHANGE [$$65(ASC) ]  |PARTITIONED|
+          limit 3 [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
+          -- STREAM_LIMIT  |PARTITIONED|
+            exchange [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              order (topK: 3) (ASC, $$65) [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
+              -- STABLE_SORT [topK: 3] [$$65(ASC)]  |PARTITIONED|
+                exchange [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  assign [$$69] <- [$$d.getField(1)] project: [$$65, $$69] [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
+                  -- ASSIGN  |PARTITIONED|
+                    exchange [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$65, $$d] <- test.DatasetWithKnownField [cardinality: 20.0, op-cost: 20.0, total-cost: 20.0]
+                      -- DATASOURCE_SCAN  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$65, $$d] <- test.DatasetWithKnownField [cardinality: 20.0, op-cost: 20.0, total-cost: 20.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|
+                          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_cbo/view/view-pushdown/view-pushdown.12.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/view/view-pushdown/view-pushdown.12.plan
index 40bee5c..a25bb52 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/view/view-pushdown/view-pushdown.12.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/view/view-pushdown/view-pushdown.12.plan
@@ -2,31 +2,27 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 35.51]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$94]) [cardinality: 6.0, op-cost: 0.0, total-cost: 35.51]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$94] <- [{"id": $$97, "review": $$101}] [cardinality: 6.0, op-cost: 0.0, total-cost: 35.51]
-      -- ASSIGN  |UNPARTITIONED|
-        exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 35.51]
-        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-          order (ASC, $$97) [cardinality: 6.0, op-cost: 15.51, total-cost: 35.51]
-          -- STABLE_SORT [$$97(ASC)]  |UNPARTITIONED|
-            exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
-            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-              limit 3 [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
-              -- STREAM_LIMIT  |UNPARTITIONED|
-                exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
-                -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                  limit 3 [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
-                  -- STREAM_LIMIT  |PARTITIONED|
-                    project ([$$97, $$101]) [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$101] <- [$$d.getField(1)] [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
-                      -- ASSIGN  |PARTITIONED|
-                        exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
+    assign [$$94] <- [{"id": $$97, "review": $$101}] project: [$$94] [cardinality: 6.0, op-cost: 0.0, total-cost: 35.51]
+    -- ASSIGN  |UNPARTITIONED|
+      exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 35.51]
+      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+        order (ASC, $$97) [cardinality: 6.0, op-cost: 15.51, total-cost: 35.51]
+        -- STABLE_SORT [$$97(ASC)]  |UNPARTITIONED|
+          exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
+          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+            limit 3 [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
+            -- STREAM_LIMIT  |UNPARTITIONED|
+              exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
+              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                limit 3 [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
+                -- STREAM_LIMIT  |PARTITIONED|
+                  assign [$$101] <- [$$d.getField(1)] project: [$$97, $$101] [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
+                  -- ASSIGN  |PARTITIONED|
+                    exchange [cardinality: 6.0, op-cost: 0.0, total-cost: 20.0]
+                    -- 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: 6.0, op-cost: 20.0, total-cost: 20.0]
+                      -- DATASOURCE_SCAN  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- 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: 6.0, op-cost: 20.0, total-cost: 20.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|
+                          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_cbo/view/view-pushdown/view-pushdown.14.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/view/view-pushdown/view-pushdown.14.plan
index 2913174..f1798fd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/view/view-pushdown/view-pushdown.14.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cbo/view/view-pushdown/view-pushdown.14.plan
@@ -4,29 +4,25 @@
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit 3 [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$69]) [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
-      -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$69] <- [{"id": $$71, "review": $$75}] [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
-        -- ASSIGN  |PARTITIONED|
-          exchange [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
-          -- SORT_MERGE_EXCHANGE [$$71(ASC) ]  |PARTITIONED|
-            limit 3 [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
-            -- STREAM_LIMIT  |PARTITIONED|
-              exchange [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (topK: 3) (ASC, $$71) [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
-                -- STABLE_SORT [topK: 3] [$$71(ASC)]  |PARTITIONED|
-                  exchange [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$71, $$75]) [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$75] <- [$$d.getField(1)] [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
-                      -- ASSIGN  |PARTITIONED|
-                        exchange [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
+      assign [$$69] <- [{"id": $$71, "review": $$75}] project: [$$69] [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
+      -- ASSIGN  |PARTITIONED|
+        exchange [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
+        -- SORT_MERGE_EXCHANGE [$$71(ASC) ]  |PARTITIONED|
+          limit 3 [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
+          -- STREAM_LIMIT  |PARTITIONED|
+            exchange [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              order (topK: 3) (ASC, $$71) [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
+              -- STABLE_SORT [topK: 3] [$$71(ASC)]  |PARTITIONED|
+                exchange [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  assign [$$75] <- [$$d.getField(1)] project: [$$71, $$75] [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
+                  -- ASSIGN  |PARTITIONED|
+                    exchange [cardinality: 20.0, op-cost: 0.0, total-cost: 20.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$71, $$d] <- test.DatasetWithKnownField [cardinality: 20.0, op-cost: 20.0, total-cost: 20.0]
+                      -- DATASOURCE_SCAN  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$71, $$d] <- test.DatasetWithKnownField [cardinality: 20.0, op-cost: 20.0, total-cost: 20.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|
+                          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_cloud/ddl/create_dataset_with_filter_on_meta/create_dataset_with_filter_on_meta.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results_cloud/ddl/create_dataset_with_filter_on_meta/create_dataset_with_filter_on_meta.1.adm
index e460060..1f0dd69 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cloud/ddl/create_dataset_with_filter_on_meta/create_dataset_with_filter_on_meta.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cloud/ddl/create_dataset_with_filter_on_meta/create_dataset_with_filter_on_meta.1.adm
@@ -1,3 +1,3 @@
-{ "DatabaseName": "Default", "DataverseName": "KeyVerse", "DatasetName": "KVStore", "IndexName": "KVStore", "IndexStructure": "BTREE", "SearchKey": [ [ "key" ] ], "IsPrimary": true, "Timestamp": "Sat Sep 30 23:12:54 PDT 2023", "PendingOp": 0, "SearchKeySourceIndicator": [ 1 ] }
-{ "DatabaseName": "Default", "DataverseName": "KeyVerse", "DatasetName": "KVStore", "IndexName": "s_location", "IndexStructure": "RTREE", "SearchKey": [ [ "location" ] ], "IsPrimary": false, "Timestamp": "Sat Sep 30 23:12:54 PDT 2023", "PendingOp": 0 }
-{ "DatabaseName": "Default", "DataverseName": "KeyVerse", "DatasetName": "KVStore", "IndexName": "s_rating", "IndexStructure": "BTREE", "SearchKey": [ [ "area_code" ] ], "IsPrimary": false, "Timestamp": "Sat Sep 30 23:12:54 PDT 2023", "PendingOp": 0, "ExcludeUnknownKey": false }
\ No newline at end of file
+{ "DatabaseName": "Default", "DataverseName": "KeyVerse", "DatasetName": "KVStore", "IndexName": "KVStore", "IndexStructure": "BTREE", "SearchKey": [ [ "key" ] ], "IsPrimary": true, "Timestamp": "Mon Oct 14 07:37:28 UTC 2024", "PendingOp": 0, "SearchKeySourceIndicator": [ 1 ], "Creator": { "Name": "@sys", "Uuid": "97c793f3-bcbf-4595-8bf0-e9d6a5953523" } }
+{ "DatabaseName": "Default", "DataverseName": "KeyVerse", "DatasetName": "KVStore", "IndexName": "s_location", "IndexStructure": "RTREE", "SearchKey": [ [ "location" ] ], "IsPrimary": false, "Timestamp": "Sat Sep 30 23:12:54 PDT 2023", "PendingOp": 0, "Creator": { "Name": "@sys", "Uuid": "97c793f3-bcbf-4595-8bf0-e9d6a5953523" } }
+{ "DatabaseName": "Default", "DataverseName": "KeyVerse", "DatasetName": "KVStore", "IndexName": "s_rating", "IndexStructure": "BTREE", "SearchKey": [ [ "area_code" ] ], "IsPrimary": false, "Timestamp": "Sat Sep 30 23:12:54 PDT 2023", "PendingOp": 0, "ExcludeUnknownKey": false, "Creator": { "Name": "@sys", "Uuid": "97c793f3-bcbf-4595-8bf0-e9d6a5953523" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_cloud/ddl/index-cast-null/index-cast-null.015.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results_cloud/ddl/index-cast-null/index-cast-null.015.adm
index 52eaa75..797b6a6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cloud/ddl/index-cast-null/index-cast-null.015.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cloud/ddl/index-cast-null/index-cast-null.015.adm
@@ -1 +1 @@
-{ "DatabaseName": "Default", "DataverseName": "test", "DatasetName": "ds2", "IndexName": "ds2_o_idx2", "IndexStructure": "BTREE", "SearchKey": [ [ "o_s_f" ], [ "o_d_f" ] ], "IsPrimary": false, "Timestamp": "Sat Sep 30 23:18:19 PDT 2023", "PendingOp": 0, "SearchKeyType": [ "double", "string" ], "ExcludeUnknownKey": false, "Cast": { "Default": null } }
\ No newline at end of file
+{ "DatabaseName": "Default", "DataverseName": "test", "DatasetName": "ds2", "IndexName": "ds2_o_idx2", "IndexStructure": "BTREE", "SearchKey": [ [ "o_s_f" ], [ "o_d_f" ] ], "IsPrimary": false, "Timestamp": "Mon Oct 14 07:37:29 UTC 2024", "PendingOp": 0, "SearchKeyType": [ "double", "string" ], "ExcludeUnknownKey": false, "Cast": { "Default": null }, "Creator": { "Name": "@sys", "Uuid": "97c793f3-bcbf-4595-8bf0-e9d6a5953523" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_cloud/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results_cloud/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.7.adm
index 0003857..73c6eb5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cloud/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.7.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cloud/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.7.adm
@@ -1,4 +1,4 @@
-{ "DatabaseName": "Default", "DataverseName": "MyDataVerse", "DatasetName": "MyMessageDataset", "IndexName": "MyMessageDataset", "IndexStructure": "BTREE", "SearchKey": [ [ "myMessageId" ] ], "IsPrimary": true, "Timestamp": "Sat Sep 30 23:33:32 PDT 2023", "PendingOp": 0 }
-{ "DatabaseName": "Default", "DataverseName": "MyDataVerse", "DatasetName": "MyMessageDataset", "IndexName": "message_ft_index_0", "IndexStructure": "SINGLE_PARTITION_WORD_INVIX", "SearchKey": [ [ "myMessageBody" ] ], "IsPrimary": false, "Timestamp": "Sat Sep 30 23:33:33 PDT 2023", "PendingOp": 0 }
-{ "DatabaseName": "Default", "DataverseName": "MyDataVerse", "DatasetName": "MyMessageDataset", "IndexName": "message_ft_index_1", "IndexStructure": "SINGLE_PARTITION_WORD_INVIX", "SearchKey": [ [ "myMessageBody" ] ], "IsPrimary": false, "Timestamp": "Sat Sep 30 23:33:34 PDT 2023", "PendingOp": 0, "FullTextConfig": "my_first_stopword_config" }
-{ "DatabaseName": "Default", "DataverseName": "MyDataVerse", "DatasetName": "MyMessageDataset", "IndexName": "message_ft_index_2", "IndexStructure": "SINGLE_PARTITION_WORD_INVIX", "SearchKey": [ [ "myMessageBody" ] ], "IsPrimary": false, "Timestamp": "Sat Sep 30 23:33:34 PDT 2023", "PendingOp": 0, "FullTextConfig": "my_second_stopword_config" }
\ No newline at end of file
+{ "DatabaseName": "Default", "DataverseName": "MyDataVerse", "DatasetName": "MyMessageDataset", "IndexName": "MyMessageDataset", "IndexStructure": "BTREE", "SearchKey": [ [ "myMessageId" ] ], "IsPrimary": true, "Timestamp": "Wed Oct 09 14:02:40 IST 2024", "PendingOp": 0, "Creator": { "Name": "@sys", "Uuid": "97c793f3-bcbf-4595-8bf0-e9d6a5953523" } }
+{ "DatabaseName": "Default", "DataverseName": "MyDataVerse", "DatasetName": "MyMessageDataset", "IndexName": "message_ft_index_0", "IndexStructure": "SINGLE_PARTITION_WORD_INVIX", "SearchKey": [ [ "myMessageBody" ] ], "IsPrimary": false, "Timestamp": "Wed Oct 09 14:04:53 IST 2024", "PendingOp": 0, "Creator": { "Name": "@sys", "Uuid": "97c793f3-bcbf-4595-8bf0-e9d6a5953523" } }
+{ "DatabaseName": "Default", "DataverseName": "MyDataVerse", "DatasetName": "MyMessageDataset", "IndexName": "message_ft_index_1", "IndexStructure": "SINGLE_PARTITION_WORD_INVIX", "SearchKey": [ [ "myMessageBody" ] ], "IsPrimary": false, "Timestamp": "Wed Oct 09 14:06:04 IST 2024", "PendingOp": 0, "FullTextConfig": "my_first_stopword_config", "Creator": { "Name": "@sys", "Uuid": "97c793f3-bcbf-4595-8bf0-e9d6a5953523" } }
+{ "DatabaseName": "Default", "DataverseName": "MyDataVerse", "DatasetName": "MyMessageDataset", "IndexName": "message_ft_index_2", "IndexStructure": "SINGLE_PARTITION_WORD_INVIX", "SearchKey": [ [ "myMessageBody" ] ], "IsPrimary": false, "Timestamp": "Wed Oct 09 14:08:21 IST 2024", "PendingOp": 0, "FullTextConfig": "my_second_stopword_config", "Creator": { "Name": "@sys", "Uuid": "97c793f3-bcbf-4595-8bf0-e9d6a5953523" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_cloud/synonym/synonym-01/synonym-01.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results_cloud/synonym/synonym-01/synonym-01.9.adm
index 6427b88..886eef8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cloud/synonym/synonym-01/synonym-01.9.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cloud/synonym/synonym-01/synonym-01.9.adm
@@ -1,4 +1,4 @@
-{ "DatabaseName": "Default", "DataverseName": "test", "SynonymName": "TweetMessagesSynonym", "ObjectDataverseName": "test", "ObjectName": "TweetMessages", "ObjectDatabaseName": "Default" }
-{ "DatabaseName": "Default", "DataverseName": "test", "SynonymName": "TweetMessagesSynonym2", "ObjectDataverseName": "test", "ObjectName": "TweetMessagesSynonym", "ObjectDatabaseName": "Default" }
-{ "DatabaseName": "Default", "DataverseName": "test", "SynonymName": "TwitterUsersSynonym", "ObjectDataverseName": "test", "ObjectName": "TwitterUsers", "ObjectDatabaseName": "Default" }
-{ "DatabaseName": "Default", "DataverseName": "test", "SynonymName": "TwitterUsersSynonym2", "ObjectDataverseName": "test", "ObjectName": "TwitterUsersSynonym", "ObjectDatabaseName": "Default" }
+{ "DatabaseName": "Default", "DataverseName": "test", "SynonymName": "TweetMessagesSynonym", "ObjectDataverseName": "test", "ObjectName": "TweetMessages", "ObjectDatabaseName": "Default", "Creator": { "Name": "@sys", "Uuid": "97c793f3-bcbf-4595-8bf0-e9d6a5953523" } }
+{ "DatabaseName": "Default", "DataverseName": "test", "SynonymName": "TweetMessagesSynonym2", "ObjectDataverseName": "test", "ObjectName": "TweetMessagesSynonym", "ObjectDatabaseName": "Default", "Creator": { "Name": "@sys", "Uuid": "97c793f3-bcbf-4595-8bf0-e9d6a5953523" } }
+{ "DatabaseName": "Default", "DataverseName": "test", "SynonymName": "TwitterUsersSynonym", "ObjectDataverseName": "test", "ObjectName": "TwitterUsers", "ObjectDatabaseName": "Default", "Creator": { "Name": "@sys", "Uuid": "97c793f3-bcbf-4595-8bf0-e9d6a5953523" } }
+{ "DatabaseName": "Default", "DataverseName": "test", "SynonymName": "TwitterUsersSynonym2", "ObjectDataverseName": "test", "ObjectName": "TwitterUsersSynonym", "ObjectDatabaseName": "Default", "Creator": { "Name": "@sys", "Uuid": "97c793f3-bcbf-4595-8bf0-e9d6a5953523" } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_cloud/user-defined-functions/udf32_metadata/udf32_metadata.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results_cloud/user-defined-functions/udf32_metadata/udf32_metadata.2.adm
index a1c751b..7556190 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_cloud/user-defined-functions/udf32_metadata/udf32_metadata.2.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_cloud/user-defined-functions/udf32_metadata/udf32_metadata.2.adm
@@ -1,8 +1,8 @@
-{ "fn": { "DatabaseName": "Default", "DataverseName": "test", "Name": "myfn001", "Arity": "0", "Params": [  ], "ReturnType": "", "Definition": "42", "Language": "SQLPP", "Kind": "SCALAR", "Dependencies": [ [  ], [  ], [  ] ] } }
-{ "fn": { "DatabaseName": "Default", "DataverseName": "test", "Name": "myfn002", "Arity": "1", "Params": [ "a" ], "ReturnType": "", "Definition": "a", "Language": "SQLPP", "Kind": "SCALAR", "Dependencies": [ [  ], [  ], [  ] ] } }
-{ "fn": { "DatabaseName": "Default", "DataverseName": "test", "Name": "myfn003", "Arity": "2", "Params": [ "a", "b" ], "ReturnType": "", "Definition": "a + b", "Language": "SQLPP", "Kind": "SCALAR", "Dependencies": [ [  ], [  ], [  ] ] } }
-{ "fn": { "DatabaseName": "Default", "DataverseName": "test", "Name": "myfn004", "Arity": "-1", "Params": [ "args" ], "ReturnType": "", "Definition": "args", "Language": "SQLPP", "Kind": "SCALAR", "Dependencies": [ [  ], [  ], [  ] ] } }
-{ "fn": { "DatabaseName": "Default", "DataverseName": "test", "Name": "myfn005", "Arity": "1", "Params": [ "a" ], "ReturnType": "", "Definition": "a", "Language": "SQLPP", "Kind": "SCALAR", "Dependencies": [ [  ], [  ], [  ] ] } }
-{ "fn": { "DatabaseName": "Default", "DataverseName": "test", "Name": "myfn005", "Arity": "2", "Params": [ "a", "b" ], "ReturnType": "", "Definition": "a + b", "Language": "SQLPP", "Kind": "SCALAR", "Dependencies": [ [  ], [  ], [  ] ] } }
-{ "fn": { "DatabaseName": "Default", "DataverseName": "test", "Name": "myfn006", "Arity": "-1", "Params": [ "args" ], "ReturnType": "", "Definition": "args", "Language": "SQLPP", "Kind": "SCALAR", "Dependencies": [ [  ], [  ], [  ] ] } }
-{ "fn": { "DatabaseName": "Default", "DataverseName": "test", "Name": "myfn006", "Arity": "1", "Params": [ "a" ], "ReturnType": "", "Definition": "a", "Language": "SQLPP", "Kind": "SCALAR", "Dependencies": [ [  ], [  ], [  ] ] } }
\ No newline at end of file
+{ "fn": { "DatabaseName": "Default", "DataverseName": "test", "Name": "myfn001", "Arity": "0", "Params": [  ], "ReturnType": "", "Definition": "42", "Language": "SQLPP", "Kind": "SCALAR", "Dependencies": [ [  ], [  ], [  ] ], "Creator": { "Name": "@sys", "Uuid": "97c793f3-bcbf-4595-8bf0-e9d6a5953523" } } }
+{ "fn": { "DatabaseName": "Default", "DataverseName": "test", "Name": "myfn002", "Arity": "1", "Params": [ "a" ], "ReturnType": "", "Definition": "a", "Language": "SQLPP", "Kind": "SCALAR", "Dependencies": [ [  ], [  ], [  ] ], "Creator": { "Name": "@sys", "Uuid": "97c793f3-bcbf-4595-8bf0-e9d6a5953523" } } }
+{ "fn": { "DatabaseName": "Default", "DataverseName": "test", "Name": "myfn003", "Arity": "2", "Params": [ "a", "b" ], "ReturnType": "", "Definition": "a + b", "Language": "SQLPP", "Kind": "SCALAR", "Dependencies": [ [  ], [  ], [  ] ], "Creator": { "Name": "@sys", "Uuid": "97c793f3-bcbf-4595-8bf0-e9d6a5953523" } } }
+{ "fn": { "DatabaseName": "Default", "DataverseName": "test", "Name": "myfn004", "Arity": "-1", "Params": [ "args" ], "ReturnType": "", "Definition": "args", "Language": "SQLPP", "Kind": "SCALAR", "Dependencies": [ [  ], [  ], [  ] ], "Creator": { "Name": "@sys", "Uuid": "97c793f3-bcbf-4595-8bf0-e9d6a5953523" } } }
+{ "fn": { "DatabaseName": "Default", "DataverseName": "test", "Name": "myfn005", "Arity": "1", "Params": [ "a" ], "ReturnType": "", "Definition": "a", "Language": "SQLPP", "Kind": "SCALAR", "Dependencies": [ [  ], [  ], [  ] ], "Creator": { "Name": "@sys", "Uuid": "97c793f3-bcbf-4595-8bf0-e9d6a5953523" } } }
+{ "fn": { "DatabaseName": "Default", "DataverseName": "test", "Name": "myfn005", "Arity": "2", "Params": [ "a", "b" ], "ReturnType": "", "Definition": "a + b", "Language": "SQLPP", "Kind": "SCALAR", "Dependencies": [ [  ], [  ], [  ] ], "Creator": { "Name": "@sys", "Uuid": "97c793f3-bcbf-4595-8bf0-e9d6a5953523" } } }
+{ "fn": { "DatabaseName": "Default", "DataverseName": "test", "Name": "myfn006", "Arity": "-1", "Params": [ "args" ], "ReturnType": "", "Definition": "args", "Language": "SQLPP", "Kind": "SCALAR", "Dependencies": [ [  ], [  ], [  ] ], "Creator": { "Name": "@sys", "Uuid": "97c793f3-bcbf-4595-8bf0-e9d6a5953523" } } }
+{ "fn": { "DatabaseName": "Default", "DataverseName": "test", "Name": "myfn006", "Arity": "1", "Params": [ "a" ], "ReturnType": "", "Definition": "a", "Language": "SQLPP", "Kind": "SCALAR", "Dependencies": [ [  ], [  ], [  ] ], "Creator": { "Name": "@sys", "Uuid": "97c793f3-bcbf-4595-8bf0-e9d6a5953523" } } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/aggregate-sql/count_dataset/count_dataset.1.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/aggregate-sql/count_dataset/count_dataset.1.plan
index ee82571..59eeaf0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/aggregate-sql/count_dataset/count_dataset.1.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/aggregate-sql/count_dataset/count_dataset.1.plan
@@ -12,19 +12,15 @@
           -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$27]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select (and(ge($$25, 1), le($$25, 10))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  project ([$$27, $$25]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$25] <- [$$Tweet.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
+              select (and(ge($$25, 1), le($$25, 10))) project: [$$27] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                assign [$$25] <- [$$Tweet.getField(1)] project: [$$27, $$25] [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]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$27, $$Tweet] <- Twitter.Tweet project ({sid:any}) filter on: and(ge($$Tweet.getField(1), 1), le($$Tweet.getField(1), 10)) range-filter on: and(ge($$Tweet.getField(1), 1), le($$Tweet.getField(1), 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|
-                        data-scan []<-[$$27, $$Tweet] <- Twitter.Tweet project ({sid:any}) filter on: and(ge($$Tweet.getField(1), 1), le($$Tweet.getField(1), 10)) range-filter on: and(ge($$Tweet.getField(1), 1), le($$Tweet.getField(1), 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|
+                        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/aggregate/count_dataset/count_dataset.1.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/aggregate/count_dataset/count_dataset.1.plan
index 31c4954..2107c0e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/aggregate/count_dataset/count_dataset.1.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/aggregate/count_dataset/count_dataset.1.plan
@@ -12,19 +12,15 @@
           -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$27]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select (and(ge($$25, 1), le($$25, 10))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  project ([$$27, $$25]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$25] <- [$$Tweet.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
+              select (and(ge($$25, 1), le($$25, 10))) project: [$$27] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                assign [$$25] <- [$$Tweet.getField(1)] project: [$$27, $$25] [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]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    data-scan []<-[$$27, $$Tweet] <- Twitter.Tweet project ({sid:any}) filter on: and(ge($$Tweet.getField(1), 1), le($$Tweet.getField(1), 10)) range-filter on: and(ge($$Tweet.getField(1), 1), le($$Tweet.getField(1), 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|
-                        data-scan []<-[$$27, $$Tweet] <- Twitter.Tweet project ({sid:any}) filter on: and(ge($$Tweet.getField(1), 1), le($$Tweet.getField(1), 10)) range-filter on: and(ge($$Tweet.getField(1), 1), le($$Tweet.getField(1), 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|
+                        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/cardinality-estimation/join-queries/join-queries.3.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/join-queries/join-queries.3.plan
index d90a514..74ee098 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/join-queries/join-queries.3.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/join-queries/join-queries.3.plan
@@ -10,43 +10,37 @@
         -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
-            project ([$$35]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              assign [$$35] <- [{"l_linenumber": $$37, "l_partkey": $$43, "o_custkey": $$44}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ASSIGN  |PARTITIONED|
-                project ([$$37, $$43, $$44]) [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|
-                    join (eq($$36, $$38)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HYBRID_HASH_JOIN [$$36][$$38]  |PARTITIONED|
-                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
-                        project ([$$37, $$43, $$36]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          select (gt($$l.getField(4), 10)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- STREAM_SELECT  |PARTITIONED|
-                            assign [$$43] <- [$$l.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- ASSIGN  |PARTITIONED|
+            assign [$$35] <- [{"l_linenumber": $$37, "l_partkey": $$43, "o_custkey": $$44}] project: [$$35] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ASSIGN  |PARTITIONED|
+              project ([$$37, $$43, $$44]) [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|
+                  join (eq($$36, $$38)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HYBRID_HASH_JOIN [$$36][$$38]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
+                      select (gt($$l.getField(4), 10)) project: [$$37, $$43, $$36] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- STREAM_SELECT  |PARTITIONED|
+                        assign [$$43] <- [$$l.getField(1)] [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]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            data-scan []<-[$$36, $$37, $$l] <- tpch.LineItem project ({l_partkey:any,l_quantity:any}) filter on: gt($$l.getField(4), 10) range-filter on: gt($$l.getField(4), 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|
-                                data-scan []<-[$$36, $$37, $$l] <- tpch.LineItem project ({l_partkey:any,l_quantity:any}) filter on: gt($$l.getField(4), 10) range-filter on: gt($$l.getField(4), 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]
-                      -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
-                        project ([$$44, $$38]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$44] <- [$$o.getField(1)] [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|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                      assign [$$44] <- [$$o.getField(1)] project: [$$44, $$38] [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]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$38, $$o] <- tpch.Orders project ({o_custkey:any}) [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|
-                              data-scan []<-[$$38, $$o] <- tpch.Orders project ({o_custkey:any}) [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|
+                              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/cardinality-estimation/join-queries/join-queries.4.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/join-queries/join-queries.4.plan
index 91ec187..b201286 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/join-queries/join-queries.4.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/join-queries/join-queries.4.plan
@@ -10,43 +10,37 @@
         -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
           -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
-            project ([$$34]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STREAM_PROJECT  |PARTITIONED|
-              assign [$$34] <- [{"l_orderkey": $$36, "o_custkey": $$41}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ASSIGN  |PARTITIONED|
-                project ([$$36, $$41]) [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|
-                    join (eq($$36, $$38)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HYBRID_HASH_JOIN [$$36][$$38]  |PARTITIONED|
-                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
-                        project ([$$36]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            assign [$$34] <- [{"l_orderkey": $$36, "o_custkey": $$41}] project: [$$34] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ASSIGN  |PARTITIONED|
+              project ([$$36, $$41]) [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|
+                  join (eq($$36, $$38)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HYBRID_HASH_JOIN [$$36][$$38]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
+                      select (gt($$l.getField(1), 5)) project: [$$36] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- STREAM_SELECT  |PARTITIONED|
+                        project ([$$36, $$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- STREAM_PROJECT  |PARTITIONED|
-                          select (gt($$l.getField(1), 5)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- STREAM_SELECT  |PARTITIONED|
-                            project ([$$36, $$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 []<-[$$36, $$37, $$l] <- tpch.LineItem project ({l_partkey:any}) filter on: gt($$l.getField(1), 5) range-filter on: gt($$l.getField(1), 5) [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|
-                                data-scan []<-[$$36, $$37, $$l] <- tpch.LineItem project ({l_partkey:any}) filter on: gt($$l.getField(1), 5) range-filter on: gt($$l.getField(1), 5) [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 [$$38]  |PARTITIONED|
-                        project ([$$41, $$38]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$41] <- [$$o.getField(1)] [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|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                      assign [$$41] <- [$$o.getField(1)] project: [$$41, $$38] [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]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$38, $$o] <- tpch.Orders project ({o_custkey:any}) [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|
-                              data-scan []<-[$$38, $$o] <- tpch.Orders project ({o_custkey:any}) [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|
+                              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/cardinality-estimation/join-queries/join-queries.5.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/join-queries/join-queries.5.plan
index 4a9bbcc..dc90a31 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/join-queries/join-queries.5.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/join-queries/join-queries.5.plan
@@ -2,87 +2,79 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$101]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$101] <- [{"$1": $$111}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$111]) [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|
-            group by ([$$l_partkey := $$115; $$o_orderstatus := $$116; $$c_nationkey := $$117]) decor ([]) {
-                      aggregate [$$111] <- [agg-sql-sum($$114)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- AGGREGATE  |LOCAL|
-                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- SORT_GROUP_BY[$$115, $$116, $$117]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HASH_PARTITION_EXCHANGE [$$115, $$116, $$117]  |PARTITIONED|
-                group by ([$$115 := $$102; $$116 := $$103; $$117 := $$104]) decor ([]) {
-                          aggregate [$$114] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- SORT_GROUP_BY[$$102, $$103, $$104]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$102, $$103, $$104]) [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|
-                        join (eq($$109, $$108)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HYBRID_HASH_JOIN [$$109][$$108]  |PARTITIONED|
-                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- HASH_PARTITION_EXCHANGE [$$109]  |PARTITIONED|
-                            project ([$$102, $$103, $$109]) [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|
-                                join (eq($$105, $$107)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- HYBRID_HASH_JOIN [$$105][$$107]  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- HASH_PARTITION_EXCHANGE [$$105]  |PARTITIONED|
-                                    project ([$$102, $$105]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$101] <- [{"$1": $$111}] project: [$$101] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$111]) [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|
+          group by ([$$l_partkey := $$115; $$o_orderstatus := $$116; $$c_nationkey := $$117]) decor ([]) {
+                    aggregate [$$111] <- [agg-sql-sum($$114)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- SORT_GROUP_BY[$$115, $$116, $$117]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HASH_PARTITION_EXCHANGE [$$115, $$116, $$117]  |PARTITIONED|
+              group by ([$$115 := $$102; $$116 := $$103; $$117 := $$104]) decor ([]) {
+                        aggregate [$$114] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- AGGREGATE  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- SORT_GROUP_BY[$$102, $$103, $$104]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  project ([$$102, $$103, $$104]) [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|
+                      join (eq($$109, $$108)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- HYBRID_HASH_JOIN [$$109][$$108]  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- HASH_PARTITION_EXCHANGE [$$109]  |PARTITIONED|
+                          project ([$$102, $$103, $$109]) [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|
+                              join (eq($$105, $$107)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- HYBRID_HASH_JOIN [$$105][$$107]  |PARTITIONED|
+                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- HASH_PARTITION_EXCHANGE [$$105]  |PARTITIONED|
+                                  assign [$$102] <- [$$l.getField(1)] project: [$$102, $$105] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- ASSIGN  |PARTITIONED|
+                                    project ([$$105, $$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                     -- STREAM_PROJECT  |PARTITIONED|
-                                      assign [$$102] <- [$$l.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- ASSIGN  |PARTITIONED|
-                                        project ([$$105, $$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 []<-[$$105, $$106, $$l] <- tpch.LineItem project ({l_partkey:any}) [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|
-                                            data-scan []<-[$$105, $$106, $$l] <- tpch.LineItem project ({l_partkey:any}) [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 [$$107]  |PARTITIONED|
-                                    project ([$$103, $$109, $$107]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      assign [$$103, $$109] <- [$$o.getField(2), $$o.getField(1)] [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]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          data-scan []<-[$$107, $$o] <- tpch.Orders project ({o_orderstatus:any,o_custkey:any}) [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 [$$108]  |PARTITIONED|
-                            project ([$$104, $$108]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              assign [$$104] <- [$$c.getField(3)] [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|
                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  data-scan []<-[$$108, $$c] <- tpch.Customer project ({c_nationkey:any}) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$107]  |PARTITIONED|
+                                  assign [$$103, $$109] <- [$$o.getField(2), $$o.getField(1)] project: [$$103, $$109, $$107] [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]
                                     -- 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 []<-[$$107, $$o] <- tpch.Orders project ({o_orderstatus:any,o_custkey:any}) [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 [$$108]  |PARTITIONED|
+                          assign [$$104] <- [$$c.getField(3)] project: [$$104, $$108] [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]
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              data-scan []<-[$$108, $$c] <- tpch.Customer project ({c_nationkey:any}) [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_column/cardinality-estimation/join-queries/join-queries.6.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/join-queries/join-queries.6.plan
index d6cd212..65a2966 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/join-queries/join-queries.6.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/join-queries/join-queries.6.plan
@@ -2,83 +2,77 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$101]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$101] <- [{"$1": $$111}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$111]) [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|
-            group by ([$$l_linenumber := $$115; $$o_orderstatus := $$116; $$c_nationkey := $$117]) decor ([]) {
-                      aggregate [$$111] <- [agg-sql-sum($$114)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- AGGREGATE  |LOCAL|
-                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- SORT_GROUP_BY[$$115, $$116, $$117]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HASH_PARTITION_EXCHANGE [$$115, $$116, $$117]  |PARTITIONED|
-                group by ([$$115 := $$106; $$116 := $$103; $$117 := $$104]) decor ([]) {
-                          aggregate [$$114] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- SORT_GROUP_BY[$$106, $$103, $$104]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$106, $$103, $$104]) [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|
-                        join (eq($$109, $$108)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HYBRID_HASH_JOIN [$$109][$$108]  |PARTITIONED|
-                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- HASH_PARTITION_EXCHANGE [$$109]  |PARTITIONED|
-                            project ([$$106, $$103, $$109]) [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|
-                                join (eq($$105, $$107)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- HYBRID_HASH_JOIN [$$105][$$107]  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- HASH_PARTITION_EXCHANGE [$$105]  |PARTITIONED|
-                                    project ([$$106, $$105]) [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 []<-[$$105, $$106, $$l] <- tpch.LineItem project ({}) [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 [$$107]  |PARTITIONED|
-                                    project ([$$103, $$109, $$107]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      assign [$$103, $$109] <- [$$o.getField(2), $$o.getField(1)] [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]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          data-scan []<-[$$107, $$o] <- tpch.Orders project ({o_orderstatus:any,o_custkey:any}) [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 [$$108]  |PARTITIONED|
-                            project ([$$104, $$108]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              assign [$$104] <- [$$c.getField(3)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ASSIGN  |PARTITIONED|
+    assign [$$101] <- [{"$1": $$111}] project: [$$101] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$111]) [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|
+          group by ([$$l_linenumber := $$115; $$o_orderstatus := $$116; $$c_nationkey := $$117]) decor ([]) {
+                    aggregate [$$111] <- [agg-sql-sum($$114)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- SORT_GROUP_BY[$$115, $$116, $$117]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HASH_PARTITION_EXCHANGE [$$115, $$116, $$117]  |PARTITIONED|
+              group by ([$$115 := $$106; $$116 := $$103; $$117 := $$104]) decor ([]) {
+                        aggregate [$$114] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- AGGREGATE  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- SORT_GROUP_BY[$$106, $$103, $$104]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  project ([$$106, $$103, $$104]) [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|
+                      join (eq($$109, $$108)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- HYBRID_HASH_JOIN [$$109][$$108]  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- HASH_PARTITION_EXCHANGE [$$109]  |PARTITIONED|
+                          project ([$$106, $$103, $$109]) [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|
+                              join (eq($$105, $$107)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- HYBRID_HASH_JOIN [$$105][$$107]  |PARTITIONED|
                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  data-scan []<-[$$108, $$c] <- tpch.Customer project ({c_nationkey:any}) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$105]  |PARTITIONED|
+                                  project ([$$106, $$105]) [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|
-                                      empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      data-scan []<-[$$105, $$106, $$l] <- tpch.LineItem project ({}) [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 [$$107]  |PARTITIONED|
+                                  assign [$$103, $$109] <- [$$o.getField(2), $$o.getField(1)] project: [$$103, $$109, $$107] [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]
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      data-scan []<-[$$107, $$o] <- tpch.Orders project ({o_orderstatus:any,o_custkey:any}) [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 [$$108]  |PARTITIONED|
+                          assign [$$104] <- [$$c.getField(3)] project: [$$104, $$108] [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]
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              data-scan []<-[$$108, $$c] <- tpch.Customer project ({c_nationkey:any}) [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_column/cardinality-estimation/join-queries/join-queries.7.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/join-queries/join-queries.7.plan
index 985f3da..ba12809 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/join-queries/join-queries.7.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/join-queries/join-queries.7.plan
@@ -2,83 +2,77 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$101]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$101] <- [{"$1": $$111}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$111]) [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|
-            group by ([$$l_linenumber := $$115; $$c_nationkey := $$116; $$o_orderstatus := $$117]) decor ([]) {
-                      aggregate [$$111] <- [agg-sql-sum($$114)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- AGGREGATE  |LOCAL|
-                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- SORT_GROUP_BY[$$115, $$116, $$117]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HASH_PARTITION_EXCHANGE [$$115, $$116, $$117]  |PARTITIONED|
-                group by ([$$115 := $$106; $$116 := $$103; $$117 := $$104]) decor ([]) {
-                          aggregate [$$114] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- SORT_GROUP_BY[$$106, $$103, $$104]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$106, $$103, $$104]) [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|
-                        join (eq($$109, $$108)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HYBRID_HASH_JOIN [$$109][$$108]  |PARTITIONED|
-                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- HASH_PARTITION_EXCHANGE [$$109]  |PARTITIONED|
-                            project ([$$106, $$104, $$109]) [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|
-                                join (eq($$105, $$107)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- HYBRID_HASH_JOIN [$$105][$$107]  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- HASH_PARTITION_EXCHANGE [$$105]  |PARTITIONED|
-                                    project ([$$106, $$105]) [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 []<-[$$105, $$106, $$l] <- tpch.LineItem project ({}) [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 [$$107]  |PARTITIONED|
-                                    project ([$$104, $$109, $$107]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      assign [$$104, $$109] <- [$$o.getField(2), $$o.getField(1)] [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]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          data-scan []<-[$$107, $$o] <- tpch.Orders project ({o_orderstatus:any,o_custkey:any}) [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 [$$108]  |PARTITIONED|
-                            project ([$$103, $$108]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              assign [$$103] <- [$$c.getField(3)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ASSIGN  |PARTITIONED|
+    assign [$$101] <- [{"$1": $$111}] project: [$$101] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$111]) [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|
+          group by ([$$l_linenumber := $$115; $$c_nationkey := $$116; $$o_orderstatus := $$117]) decor ([]) {
+                    aggregate [$$111] <- [agg-sql-sum($$114)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- SORT_GROUP_BY[$$115, $$116, $$117]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HASH_PARTITION_EXCHANGE [$$115, $$116, $$117]  |PARTITIONED|
+              group by ([$$115 := $$106; $$116 := $$103; $$117 := $$104]) decor ([]) {
+                        aggregate [$$114] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- AGGREGATE  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- SORT_GROUP_BY[$$106, $$103, $$104]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  project ([$$106, $$103, $$104]) [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|
+                      join (eq($$109, $$108)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- HYBRID_HASH_JOIN [$$109][$$108]  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- HASH_PARTITION_EXCHANGE [$$109]  |PARTITIONED|
+                          project ([$$106, $$104, $$109]) [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|
+                              join (eq($$105, $$107)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- HYBRID_HASH_JOIN [$$105][$$107]  |PARTITIONED|
                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  data-scan []<-[$$108, $$c] <- tpch.Customer project ({c_nationkey:any}) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$105]  |PARTITIONED|
+                                  project ([$$106, $$105]) [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|
-                                      empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      data-scan []<-[$$105, $$106, $$l] <- tpch.LineItem project ({}) [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 [$$107]  |PARTITIONED|
+                                  assign [$$104, $$109] <- [$$o.getField(2), $$o.getField(1)] project: [$$104, $$109, $$107] [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]
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      data-scan []<-[$$107, $$o] <- tpch.Orders project ({o_orderstatus:any,o_custkey:any}) [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 [$$108]  |PARTITIONED|
+                          assign [$$103] <- [$$c.getField(3)] project: [$$103, $$108] [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]
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              data-scan []<-[$$108, $$c] <- tpch.Customer project ({c_nationkey:any}) [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_column/cardinality-estimation/join-queries/join-queries.8.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/join-queries/join-queries.8.plan
index f13e679..b3c4de2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/join-queries/join-queries.8.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/join-queries/join-queries.8.plan
@@ -2,133 +2,119 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$119]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$119] <- [{"n_name": $$n_name, "revenue": $$132}] [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]
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          group by ([$$n_name := $$142]) decor ([]) {
-                    aggregate [$$132] <- [agg-global-sql-sum($$141)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$142]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$142]  |PARTITIONED|
-              group by ([$$142 := $$120]) decor ([]) {
-                        aggregate [$$141] <- [agg-local-sql-sum(numeric-multiply($$139, numeric-subtract(1, $$140)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$120]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$139, $$140, $$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|
-                      join (and(eq($$130, $$128), eq($$131, $$143))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- HYBRID_HASH_JOIN [$$130, $$131][$$128, $$143]  |PARTITIONED|
-                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HASH_PARTITION_EXCHANGE [$$130, $$131]  |PARTITIONED|
-                          project ([$$139, $$140, $$130, $$131]) [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|
-                              join (eq($$133, $$127)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- HYBRID_HASH_JOIN [$$133][$$127]  |PARTITIONED|
-                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
-                                  project ([$$139, $$140, $$131, $$133]) [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|
-                                      join (eq($$125, $$124)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- HYBRID_HASH_JOIN [$$124][$$125]  |PARTITIONED|
-                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- HASH_PARTITION_EXCHANGE [$$124]  |PARTITIONED|
-                                          project ([$$131, $$124]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$119] <- [{"n_name": $$n_name, "revenue": $$132}] project: [$$119] [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]
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        group by ([$$n_name := $$142]) decor ([]) {
+                  aggregate [$$132] <- [agg-global-sql-sum($$141)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$142]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$142]  |PARTITIONED|
+            group by ([$$142 := $$120]) decor ([]) {
+                      aggregate [$$141] <- [agg-local-sql-sum(numeric-multiply($$139, numeric-subtract(1, $$140)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$120]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                project ([$$139, $$140, $$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|
+                    join (and(eq($$130, $$128), eq($$131, $$143))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HYBRID_HASH_JOIN [$$130, $$131][$$128, $$143]  |PARTITIONED|
+                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- HASH_PARTITION_EXCHANGE [$$130, $$131]  |PARTITIONED|
+                        project ([$$139, $$140, $$130, $$131]) [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|
+                            join (eq($$133, $$127)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- HYBRID_HASH_JOIN [$$133][$$127]  |PARTITIONED|
+                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
+                                project ([$$139, $$140, $$131, $$133]) [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|
+                                    join (eq($$125, $$124)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- HYBRID_HASH_JOIN [$$124][$$125]  |PARTITIONED|
+                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- HASH_PARTITION_EXCHANGE [$$124]  |PARTITIONED|
+                                        project ([$$131, $$124]) [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|
+                                            join (eq($$123, $$136)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                            -- HYBRID_HASH_JOIN [$$123][$$136]  |PARTITIONED|
+                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- HASH_PARTITION_EXCHANGE [$$123]  |PARTITIONED|
+                                                assign [$$131] <- [$$c.getField(3)] project: [$$131, $$123] [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]
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    data-scan []<-[$$123, $$c] <- tpch.Customer project ({c_nationkey:any}) [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 [$$136]  |PARTITIONED|
+                                                select (and(lt($$121, "1994-01-01"), ge($$121, "1993-01-01"))) project: [$$124, $$136] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                -- STREAM_SELECT  |PARTITIONED|
+                                                  assign [$$136, $$121] <- [$$o.getField(1), $$o.getField(4)] project: [$$124, $$136, $$121] [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]
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      data-scan []<-[$$124, $$o] <- tpch.Orders project ({o_orderdate:any,o_custkey:any}) filter on: and(lt($$o.getField(4), "1994-01-01"), ge($$o.getField(4), "1993-01-01")) range-filter on: and(lt($$o.getField(4), "1994-01-01"), ge($$o.getField(4), "1993-01-01")) [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 [$$125]  |PARTITIONED|
+                                        assign [$$140, $$139, $$133] <- [$$l.getField(6), $$l.getField(5), $$l.getField(2)] project: [$$139, $$140, $$133, $$125] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- ASSIGN  |PARTITIONED|
+                                          project ([$$125, $$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|
-                                              join (eq($$123, $$136)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- HYBRID_HASH_JOIN [$$123][$$136]  |PARTITIONED|
-                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- HASH_PARTITION_EXCHANGE [$$123]  |PARTITIONED|
-                                                  project ([$$131, $$123]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    assign [$$131] <- [$$c.getField(3)] [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]
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        data-scan []<-[$$123, $$c] <- tpch.Customer project ({c_nationkey:any}) [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 [$$136]  |PARTITIONED|
-                                                  project ([$$124, $$136]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    select (and(lt($$121, "1994-01-01"), ge($$121, "1993-01-01"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                      project ([$$124, $$136, $$121]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        assign [$$136, $$121] <- [$$o.getField(1), $$o.getField(4)] [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]
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            data-scan []<-[$$124, $$o] <- tpch.Orders project ({o_orderdate:any,o_custkey:any}) filter on: and(lt($$o.getField(4), "1994-01-01"), ge($$o.getField(4), "1993-01-01")) range-filter on: and(lt($$o.getField(4), "1994-01-01"), ge($$o.getField(4), "1993-01-01")) [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 [$$125]  |PARTITIONED|
-                                          project ([$$139, $$140, $$133, $$125]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            assign [$$140, $$139, $$133] <- [$$l.getField(6), $$l.getField(5), $$l.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- ASSIGN  |PARTITIONED|
-                                              project ([$$125, $$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- STREAM_PROJECT  |PARTITIONED|
+                                              data-scan []<-[$$125, $$126, $$l] <- tpch.LineItem project ({l_extendedprice:any,l_suppkey:any,l_discount:any}) [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|
-                                                  data-scan []<-[$$125, $$126, $$l] <- tpch.LineItem project ({l_extendedprice:any,l_suppkey:any,l_discount:any}) [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 [$$127]  |PARTITIONED|
-                                  project ([$$130, $$127]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    assign [$$130] <- [$$s.getField(3)] [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]
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        data-scan []<-[$$127, $$s] <- tpch.Supplier project ({s_nationkey:any}) [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 [$$128, $$143]  |PARTITIONED|
-                          project ([$$120, $$128, $$143]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            assign [$$143, $$120] <- [$$128, $$n.getField(1)] [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|
                               exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                data-scan []<-[$$128, $$n] <- tpch.Nation project ({n_name:any}) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- DATASOURCE_SCAN  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$127]  |PARTITIONED|
+                                assign [$$130] <- [$$s.getField(3)] project: [$$130, $$127] [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]
                                   -- 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 []<-[$$127, $$s] <- tpch.Supplier project ({s_nationkey:any}) [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 [$$128, $$143]  |PARTITIONED|
+                        assign [$$143, $$120] <- [$$128, $$n.getField(1)] project: [$$120, $$128, $$143] [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]
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            data-scan []<-[$$128, $$n] <- tpch.Nation project ({n_name:any}) [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_column/cardinality-estimation/single-collection-queries/single-collection-queries.3.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/single-collection-queries/single-collection-queries.3.plan
index 606ee38..17dc387 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/single-collection-queries/single-collection-queries.3.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/single-collection-queries/single-collection-queries.3.plan
@@ -2,45 +2,41 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$48]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$48] <- [{"$1": $$51}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$51]) [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|
-            group by ([$$o_custkey := $$54]) decor ([]) {
-                      aggregate [$$51] <- [agg-sql-sum($$53)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- AGGREGATE  |LOCAL|
-                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- SORT_GROUP_BY[$$54]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
-                group by ([$$54 := $$49]) decor ([]) {
-                          aggregate [$$53] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- SORT_GROUP_BY[$$49]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$49]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$48] <- [{"$1": $$51}] project: [$$48] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$51]) [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|
+          group by ([$$o_custkey := $$54]) decor ([]) {
+                    aggregate [$$51] <- [agg-sql-sum($$53)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- SORT_GROUP_BY[$$54]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+              group by ([$$54 := $$49]) decor ([]) {
+                        aggregate [$$53] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- AGGREGATE  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- SORT_GROUP_BY[$$49]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  assign [$$49] <- [$$o.getField(1)] project: [$$49] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ASSIGN  |PARTITIONED|
+                    project ([$$o]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                     -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$49] <- [$$o.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
-                        project ([$$o]) [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 [$$50, $$o] <- index-search("Orders", 0, "Default", "tpch", "Orders", false, false, 1, $$55, 0, false, true, false) project ({o_custkey:any}) [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|
-                            unnest-map [$$50, $$o] <- index-search("Orders", 0, "Default", "tpch", "Orders", false, false, 1, $$55, 0, false, true, false) project ({o_custkey:any}) [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 [$$55] <- [2] [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|
+                            assign [$$55] <- [2] [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_column/cardinality-estimation/single-collection-queries/single-collection-queries.4.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/single-collection-queries/single-collection-queries.4.plan
index 8431c64..c6b8455 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/single-collection-queries/single-collection-queries.4.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/single-collection-queries/single-collection-queries.4.plan
@@ -2,45 +2,41 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$48]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$48] <- [{"$1": $$51}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$51]) [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|
-            group by ([$$o_orderdate := $$54]) decor ([]) {
-                      aggregate [$$51] <- [agg-sql-sum($$53)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- AGGREGATE  |LOCAL|
-                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- SORT_GROUP_BY[$$54]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
-                group by ([$$54 := $$49]) decor ([]) {
-                          aggregate [$$53] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- SORT_GROUP_BY[$$49]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$49]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$48] <- [{"$1": $$51}] project: [$$48] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$51]) [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|
+          group by ([$$o_orderdate := $$54]) decor ([]) {
+                    aggregate [$$51] <- [agg-sql-sum($$53)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- SORT_GROUP_BY[$$54]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+              group by ([$$54 := $$49]) decor ([]) {
+                        aggregate [$$53] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- AGGREGATE  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- SORT_GROUP_BY[$$49]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  assign [$$49] <- [$$o.getField(4)] project: [$$49] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ASSIGN  |PARTITIONED|
+                    project ([$$o]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                     -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$49] <- [$$o.getField(4)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
-                        project ([$$o]) [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 [$$50, $$o] <- index-search("Orders", 0, "Default", "tpch", "Orders", false, false, 1, $$55, 0, false, true, false) project ({o_orderdate:any}) [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|
-                            unnest-map [$$50, $$o] <- index-search("Orders", 0, "Default", "tpch", "Orders", false, false, 1, $$55, 0, false, true, false) project ({o_orderdate:any}) [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 [$$55] <- [5] [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|
+                            assign [$$55] <- [5] [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_column/cardinality-estimation/single-collection-queries/single-collection-queries.5.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/single-collection-queries/single-collection-queries.5.plan
index 3be2e32..0a85b2f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/single-collection-queries/single-collection-queries.5.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/single-collection-queries/single-collection-queries.5.plan
@@ -2,43 +2,39 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$51]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$51] <- [{"$1": $$55}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$55]) [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|
-            group by ([$$l_orderkey := $$59]) decor ([]) {
-                      aggregate [$$55] <- [agg-sql-sum($$58)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- AGGREGATE  |LOCAL|
-                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- SORT_GROUP_BY[$$59]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HASH_PARTITION_EXCHANGE [$$59]  |PARTITIONED|
-                group by ([$$59 := $$53]) decor ([]) {
-                          aggregate [$$58] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- SORT_GROUP_BY[$$53]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$51] <- [{"$1": $$55}] project: [$$51] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$55]) [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|
+          group by ([$$l_orderkey := $$59]) decor ([]) {
+                    aggregate [$$55] <- [agg-sql-sum($$58)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- SORT_GROUP_BY[$$59]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HASH_PARTITION_EXCHANGE [$$59]  |PARTITIONED|
+              group by ([$$59 := $$53]) decor ([]) {
+                        aggregate [$$58] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- AGGREGATE  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- SORT_GROUP_BY[$$53]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  select (and(gt($$l.getField(4), 10), gt($$l.getField(1), 1))) project: [$$53] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_SELECT  |PARTITIONED|
+                    project ([$$53, $$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                     -- STREAM_PROJECT  |PARTITIONED|
-                      select (and(gt($$l.getField(4), 10), gt($$l.getField(1), 1))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_SELECT  |PARTITIONED|
-                        project ([$$53, $$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 []<-[$$53, $$54, $$l] <- tpch.LineItem project ({l_partkey:any,l_quantity:any}) filter on: and(gt($$l.getField(4), 10), gt($$l.getField(1), 1)) range-filter on: and(gt($$l.getField(4), 10), gt($$l.getField(1), 1)) [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|
-                            data-scan []<-[$$53, $$54, $$l] <- tpch.LineItem project ({l_partkey:any,l_quantity:any}) filter on: and(gt($$l.getField(4), 10), gt($$l.getField(1), 1)) range-filter on: and(gt($$l.getField(4), 10), gt($$l.getField(1), 1)) [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|
+                            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/cardinality-estimation/single-collection-queries/single-collection-queries.6.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/single-collection-queries/single-collection-queries.6.plan
index 602421a..6880089 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/single-collection-queries/single-collection-queries.6.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/single-collection-queries/single-collection-queries.6.plan
@@ -2,41 +2,37 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$51]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$51] <- [{"$1": $$55}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$55]) [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|
-            group by ([$$l_orderkey := $$59]) decor ([]) {
-                      aggregate [$$55] <- [agg-sql-sum($$58)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- AGGREGATE  |LOCAL|
-                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- SORT_GROUP_BY[$$59]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HASH_PARTITION_EXCHANGE [$$59]  |PARTITIONED|
-                group by ([$$59 := $$53]) decor ([]) {
-                          aggregate [$$58] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- SORT_GROUP_BY[$$53]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      select (and(le($$54, 4), gt($$l.getField(4), 10))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_SELECT  |PARTITIONED|
+    assign [$$51] <- [{"$1": $$55}] project: [$$51] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$55]) [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|
+          group by ([$$l_orderkey := $$59]) decor ([]) {
+                    aggregate [$$55] <- [agg-sql-sum($$58)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- SORT_GROUP_BY[$$59]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HASH_PARTITION_EXCHANGE [$$59]  |PARTITIONED|
+              group by ([$$59 := $$53]) decor ([]) {
+                        aggregate [$$58] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- AGGREGATE  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- SORT_GROUP_BY[$$53]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  select (and(le($$54, 4), gt($$l.getField(4), 10))) project: [$$53] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_SELECT  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$53, $$54, $$l] <- tpch.LineItem project ({l_quantity:any}) filter on: gt($$l.getField(4), 10) range-filter on: gt($$l.getField(4), 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|
-                          data-scan []<-[$$53, $$54, $$l] <- tpch.LineItem project ({l_quantity:any}) filter on: gt($$l.getField(4), 10) range-filter on: gt($$l.getField(4), 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|
+                          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/cardinality-estimation/single-collection-queries/single-collection-queries.7.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/single-collection-queries/single-collection-queries.7.plan
index 37871e6..50ac4cc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/single-collection-queries/single-collection-queries.7.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/cardinality-estimation/single-collection-queries/single-collection-queries.7.plan
@@ -2,47 +2,41 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$51]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$51] <- [{"$1": $$55}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$55]) [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|
-            group by ([$$l_partkey := $$59]) decor ([]) {
-                      aggregate [$$55] <- [agg-sql-sum($$58)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- AGGREGATE  |LOCAL|
-                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- SORT_GROUP_BY[$$59]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HASH_PARTITION_EXCHANGE [$$59]  |PARTITIONED|
-                group by ([$$59 := $$52]) decor ([]) {
-                          aggregate [$$58] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- SORT_GROUP_BY[$$52]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$52] <- [$$l.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
-                        project ([$$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          select (and(gt($$53, 1), gt($$54, 4))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- STREAM_SELECT  |PARTITIONED|
-                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              unnest-map [$$53, $$54, $$l] <- index-search("LineItem", 0, "Default", "tpch", "LineItem", false, false, 2, $$60, $$61, 0, true, true, false) project ({l_partkey:any}) [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 [$$60, $$61] <- [1, 4] [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|
+    assign [$$51] <- [{"$1": $$55}] project: [$$51] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$55]) [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|
+          group by ([$$l_partkey := $$59]) decor ([]) {
+                    aggregate [$$55] <- [agg-sql-sum($$58)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- SORT_GROUP_BY[$$59]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HASH_PARTITION_EXCHANGE [$$59]  |PARTITIONED|
+              group by ([$$59 := $$52]) decor ([]) {
+                        aggregate [$$58] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- AGGREGATE  |LOCAL|
+                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- SORT_GROUP_BY[$$52]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  assign [$$52] <- [$$l.getField(1)] project: [$$52] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ASSIGN  |PARTITIONED|
+                    select (and(gt($$53, 1), gt($$54, 4))) project: [$$l] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- STREAM_SELECT  |PARTITIONED|
+                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        unnest-map [$$53, $$54, $$l] <- index-search("LineItem", 0, "Default", "tpch", "LineItem", false, false, 2, $$60, $$61, 0, true, true, false) project ({l_partkey:any}) [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 [$$60, $$61] <- [1, 4] [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_column/explain/explain_field_access/explain_field_access.1.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/explain/explain_field_access/explain_field_access.1.plan
index fd69538..b97eae0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/explain/explain_field_access/explain_field_access.1.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/explain/explain_field_access/explain_field_access.1.plan
@@ -2,41 +2,37 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$50]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$50] <- [{"deptId": $#1, "star_cost": $$53}] [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]
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          group by ([$#1 := $$58]) decor ([]) {
-                    aggregate [$$53] <- [agg-global-sql-sum($$57)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$58]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-              group by ([$$58 := $$51]) decor ([]) {
-                        aggregate [$$57] <- [agg-local-sql-sum($$48)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$51]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$48, $$51]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$50] <- [{"deptId": $#1, "star_cost": $$53}] project: [$$50] [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]
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        group by ([$#1 := $$58]) decor ([]) {
+                  aggregate [$$53] <- [agg-global-sql-sum($$57)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$58]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
+            group by ([$$58 := $$51]) decor ([]) {
+                      aggregate [$$57] <- [agg-local-sql-sum($$48)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$51]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                assign [$$51, $$48] <- [substring($$e.getField("dept").getField("department_id"), 0), $$e.getField("salary")] project: [$$48, $$51] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$e]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$51, $$48] <- [substring($$e.getField("dept").getField("department_id"), 0), $$e.getField("salary")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
-                      project ([$$e]) [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 []<-[$$52, $$e] <- gby.Employee project ({dept:{department_id:any},salary:any}) [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|
-                          data-scan []<-[$$52, $$e] <- gby.Employee project ({dept:{department_id:any},salary:any}) [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|
+                          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/explain/explain_field_access_closed/explain_field_access_closed.1.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/explain/explain_field_access_closed/explain_field_access_closed.1.plan
index dc290f4..280b5c4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/explain/explain_field_access_closed/explain_field_access_closed.1.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/explain/explain_field_access_closed/explain_field_access_closed.1.plan
@@ -2,41 +2,37 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$49]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$49] <- [{"deptId": $#1, "star_cost": $$52}] [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]
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          group by ([$#1 := $$56]) decor ([]) {
-                    aggregate [$$52] <- [agg-global-sql-sum($$55)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$56]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
-              group by ([$$56 := $$50]) decor ([]) {
-                        aggregate [$$55] <- [agg-local-sql-sum($$47)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$50]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$47, $$50]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$49] <- [{"deptId": $#1, "star_cost": $$52}] project: [$$49] [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]
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        group by ([$#1 := $$56]) decor ([]) {
+                  aggregate [$$52] <- [agg-global-sql-sum($$55)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$56]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
+            group by ([$$56 := $$50]) decor ([]) {
+                      aggregate [$$55] <- [agg-local-sql-sum($$47)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$50]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                assign [$$50, $$47] <- [substring($$e.getField(1), 0), $$e.getField(2)] project: [$$47, $$50] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$e]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$50, $$47] <- [substring($$e.getField(1), 0), $$e.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
-                      project ([$$e]) [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 []<-[$$51, $$e] <- gby.Employee project ({department_id:any,salary:any}) [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|
-                          data-scan []<-[$$51, $$e] <- gby.Employee project ({department_id:any,salary:any}) [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|
+                          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/index-selection/like-expression/like-expression.04.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/index-selection/like-expression/like-expression.04.plan
new file mode 100644
index 0000000..cd9242c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/index-selection/like-expression/like-expression.04.plan
@@ -0,0 +1,34 @@
+distribute result [$$19] [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|
+    assign [$$19] <- [{"ds1": $$ds1}] project: [$$19] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (and(ge($$20, "fsdxv😀"), lt($$20, "fsdxv😁"))) project: [$$ds1] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        assign [$$20] <- [$$ds1.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- ASSIGN  |PARTITIONED|
+          project ([$$ds1]) [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, $$ds1] <- index-search("ds1", 0, "Default", "test", "ds1", false, false, 1, $$25, 1, $$25, true, true, true) range-filter on: and(ge($$ds1.getField(1), "fsdxv😀"), lt($$ds1.getField(1), "fsdxv😁")) [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, $$25) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      project ([$$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 [$$24, $$25] <- index-search("ds1_name", 0, "Default", "test", "ds1", false, false, 1, $$22, 1, $$23, 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 [$$22, $$23] <- ["fsdxv😀", "fsdxv😁"] [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_column/index-selection/like-expression/like-expression.06.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/index-selection/like-expression/like-expression.06.plan
new file mode 100644
index 0000000..7f021bb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/index-selection/like-expression/like-expression.06.plan
@@ -0,0 +1,34 @@
+distribute result [$$19] [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|
+    assign [$$19] <- [{"ds1": $$ds1}] project: [$$19] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (and(ge($$20, "asdgdh👩‍👩‍👧"), lt($$20, "asdgdh👩‍👩‍👨"))) project: [$$ds1] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        assign [$$20] <- [$$ds1.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- ASSIGN  |PARTITIONED|
+          project ([$$ds1]) [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, $$ds1] <- index-search("ds1", 0, "Default", "test", "ds1", false, false, 1, $$25, 1, $$25, true, true, true) range-filter on: and(ge($$ds1.getField(1), "asdgdh👩‍👩‍👧"), lt($$ds1.getField(1), "asdgdh👩‍👩‍👨")) [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, $$25) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      project ([$$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 [$$24, $$25] <- index-search("ds1_name", 0, "Default", "test", "ds1", false, false, 1, $$22, 1, $$23, 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 [$$22, $$23] <- ["asdgdh👩‍👩‍👧", "asdgdh👩‍👩‍👨"] [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_column/index-selection/like-expression/like-expression.08.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/index-selection/like-expression/like-expression.08.plan
new file mode 100644
index 0000000..bd94ced
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/index-selection/like-expression/like-expression.08.plan
@@ -0,0 +1,32 @@
+distribute result [$$16] [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|
+    assign [$$16] <- [{"ds1": $$ds1}] project: [$$16] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (eq($$ds1.getField(1), "sxvciis")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$ds1]) [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, $$ds1] <- index-search("ds1", 0, "Default", "test", "ds1", false, false, 1, $$22, 1, $$22, true, true, true) range-filter on: eq($$ds1.getField(1), "sxvciis") [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, $$22) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    project ([$$22]) [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] <- index-search("ds1_name", 0, "Default", "test", "ds1", false, false, 1, $$19, 1, $$20, true, true, true) [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, $$20] <- ["sxvciis", "sxvciis"] [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_column/index-selection/like-expression/like-expression.10.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/index-selection/like-expression/like-expression.10.plan
new file mode 100644
index 0000000..7f5893e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/index-selection/like-expression/like-expression.10.plan
@@ -0,0 +1,34 @@
+distribute result [$$19] [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|
+    assign [$$19] <- [{"ds1": $$ds1}] project: [$$19] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (and(ge($$20, "Joh"), lt($$20, "Joi"))) project: [$$ds1] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        assign [$$20] <- [$$ds1.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- ASSIGN  |PARTITIONED|
+          project ([$$ds1]) [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, $$ds1] <- index-search("ds1", 0, "Default", "test", "ds1", false, false, 1, $$25, 1, $$25, true, true, true) range-filter on: and(ge($$ds1.getField(1), "Joh"), lt($$ds1.getField(1), "Joi")) [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, $$25) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      project ([$$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 [$$24, $$25] <- index-search("ds1_name", 0, "Default", "test", "ds1", false, false, 1, $$22, 1, $$23, 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 [$$22, $$23] <- ["Joh", "Joi"] [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_column/index-selection/like-expression/like-expression.12.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/index-selection/like-expression/like-expression.12.plan
new file mode 100644
index 0000000..af25ff6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/index-selection/like-expression/like-expression.12.plan
@@ -0,0 +1,18 @@
+distribute result [$$16] [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|
+    assign [$$16] <- [{"ds1": $$ds1}] project: [$$16] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (like($$ds1.getField(1), "s%vfjs__jiscii%")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$ds1]) [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 []<-[$$17, $$ds1] <- test.ds1 filter on: like($$ds1.getField(1), "s%vfjs__jiscii%") [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_column/index-selection/like-expression/like-expression.14.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/index-selection/like-expression/like-expression.14.plan
new file mode 100644
index 0000000..af17a0a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/index-selection/like-expression/like-expression.14.plan
@@ -0,0 +1,18 @@
+distribute result [$$16] [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|
+    assign [$$16] <- [{"ds1": $$ds1}] project: [$$16] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (like($$ds1.getField(1), "sxvc_is")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$ds1]) [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 []<-[$$17, $$ds1] <- test.ds1 filter on: like($$ds1.getField(1), "sxvc_is") [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_column/index-selection/like-expression/like-expression.16.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/index-selection/like-expression/like-expression.16.plan
new file mode 100644
index 0000000..52e0aea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/index-selection/like-expression/like-expression.16.plan
@@ -0,0 +1,18 @@
+distribute result [$$16] [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|
+    assign [$$16] <- [{"ds1": $$ds1}] project: [$$16] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (like($$ds1.getField(1), "sxvc_i%")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$ds1]) [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 []<-[$$17, $$ds1] <- test.ds1 filter on: like($$ds1.getField(1), "sxvc_i%") [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_column/index-selection/like-expression/like-expression.18.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/index-selection/like-expression/like-expression.18.plan
new file mode 100644
index 0000000..c2401e6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/index-selection/like-expression/like-expression.18.plan
@@ -0,0 +1,18 @@
+distribute result [$$16] [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|
+    assign [$$16] <- [{"ds1": $$ds1}] project: [$$16] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (like($$ds1.getField(1), "sxvfjs%jisci%")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$ds1]) [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 []<-[$$17, $$ds1] <- test.ds1 filter on: like($$ds1.getField(1), "sxvfjs%jisci%") [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_column/index-selection/like-expression/like-expression.20.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/index-selection/like-expression/like-expression.20.plan
new file mode 100644
index 0000000..fcbf70a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/index-selection/like-expression/like-expression.20.plan
@@ -0,0 +1,34 @@
+distribute result [$$19] [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|
+    assign [$$19] <- [{"ds1": $$ds1}] project: [$$19] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (and(ge($$20, "J"), lt($$20, "K"))) project: [$$ds1] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        assign [$$20] <- [$$ds1.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- ASSIGN  |PARTITIONED|
+          project ([$$ds1]) [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, $$ds1] <- index-search("ds1", 0, "Default", "test", "ds1", false, false, 1, $$25, 1, $$25, true, true, true) range-filter on: and(ge($$ds1.getField(1), "J"), lt($$ds1.getField(1), "K")) [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, $$25) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      project ([$$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 [$$24, $$25] <- index-search("ds1_name", 0, "Default", "test", "ds1", false, false, 1, $$22, 1, $$23, 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 [$$22, $$23] <- ["J", "K"] [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_column/index-selection/like-expression/like-expression.22.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/index-selection/like-expression/like-expression.22.plan
new file mode 100644
index 0000000..ec3afe1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/index-selection/like-expression/like-expression.22.plan
@@ -0,0 +1,34 @@
+distribute result [$$19] [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|
+    assign [$$19] <- [{"ds1": $$ds1}] project: [$$19] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (and(ge($$20, "sx"), lt($$20, "sy"))) project: [$$ds1] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        assign [$$20] <- [$$ds1.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- ASSIGN  |PARTITIONED|
+          project ([$$ds1]) [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, $$ds1] <- index-search("ds1", 0, "Default", "test", "ds1", false, false, 1, $$25, 1, $$25, true, true, true) range-filter on: and(ge($$ds1.getField(1), "sx"), lt($$ds1.getField(1), "sy")) [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, $$25) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      project ([$$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 [$$24, $$25] <- index-search("ds1_name", 0, "Default", "test", "ds1", false, false, 1, $$22, 1, $$23, 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 [$$22, $$23] <- ["sx", "sy"] [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_column/index-selection/like-expression/like-expression.25.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/index-selection/like-expression/like-expression.25.plan
new file mode 100644
index 0000000..e224c27
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/index-selection/like-expression/like-expression.25.plan
@@ -0,0 +1,32 @@
+distribute result [$$16] [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|
+    assign [$$16] <- [{"ds1": $$ds1}] project: [$$16] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (eq($$ds1.getField(1), "h\axx%")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$ds1]) [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, $$ds1] <- index-search("ds1", 0, "Default", "test", "ds1", false, false, 1, $$22, 1, $$22, true, true, true) range-filter on: eq($$ds1.getField(1), "h\axx%") [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, $$22) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    project ([$$22]) [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] <- index-search("ds1_name", 0, "Default", "test", "ds1", false, false, 1, $$19, 1, $$20, true, true, true) [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, $$20] <- ["h\axx%", "h\axx%"] [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_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.04.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.04.plan
index f99a1a2..a3ad366 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.04.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.04.plan
@@ -2,63 +2,57 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$51]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$51] <- [{"n_nationkey": $$58, "s_nationkey": $$56, "c_nationkey": $$55}] [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 [$$58(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
-          order (ASC, $$58) (ASC, $$56) (ASC, $$55) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$58(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              join (eq($$55, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HYBRID_HASH_JOIN [$$58][$$55]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  join (eq($$56, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HYBRID_HASH_JOIN [$$58][$$56]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-                      project ([$$58]) [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 []<-[$$58, $$n] <- tpch.Nation project ({}) [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 [$$56]  |PARTITIONED|
-                      project ([$$56]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        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 []<-[$$59, $$s] <- tpch.Supplier project ({s_nationkey:any}) [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 [$$55]  |PARTITIONED|
-                  project ([$$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$55] <- [$$c.getField(3)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$51] <- [{"n_nationkey": $$58, "s_nationkey": $$56, "c_nationkey": $$55}] project: [$$51] [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 [$$58(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
+        order (ASC, $$58) (ASC, $$56) (ASC, $$55) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$58(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            join (eq($$55, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HYBRID_HASH_JOIN [$$58][$$55]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                join (eq($$56, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HYBRID_HASH_JOIN [$$58][$$56]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
+                    project ([$$58]) [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 []<-[$$58, $$n] <- tpch.Nation project ({}) [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 [$$56]  |PARTITIONED|
+                    assign [$$56] <- [$$s.getField(3)] project: [$$56] [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]
+                      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 []<-[$$60, $$c] <- tpch.Customer project ({c_nationkey:any}) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          data-scan []<-[$$59, $$s] <- tpch.Supplier project ({s_nationkey:any}) [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 [$$55]  |PARTITIONED|
+                assign [$$55] <- [$$c.getField(3)] project: [$$55] [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|
+                      data-scan []<-[$$60, $$c] <- tpch.Customer project ({c_nationkey:any}) [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_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.06.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.06.plan
index 16805f4..e3f4ed4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.06.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.06.plan
@@ -2,49 +2,45 @@
 -- 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]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$36] <- [{"o_orderkey": $$43, "l_orderkey": $$44, "l_suppkey": $$42}] [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 [$$43(ASC), $$44(ASC), $$42(ASC) ]  |PARTITIONED|
-          order (ASC, $$43) (ASC, $$44) (ASC, $$42) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$43(ASC), $$44(ASC), $$42(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$43, $$44, $$42]) [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|
-                  join (and(eq($$43, $$44), eq($$49, $$42))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HYBRID_HASH_JOIN [$$43, $$49][$$44, $$42]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HASH_PARTITION_EXCHANGE [$$43, $$49]  |PARTITIONED|
-                      assign [$$49] <- [$$43] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
-                        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 []<-[$$43, $$o] <- tpch.Orders project ({}) [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 [$$44, $$42]  |PARTITIONED|
-                      project ([$$44, $$42]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$36] <- [{"o_orderkey": $$43, "l_orderkey": $$44, "l_suppkey": $$42}] project: [$$36] [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 [$$43(ASC), $$44(ASC), $$42(ASC) ]  |PARTITIONED|
+        order (ASC, $$43) (ASC, $$44) (ASC, $$42) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$43(ASC), $$44(ASC), $$42(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            project ([$$43, $$44, $$42]) [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|
+                join (and(eq($$43, $$44), eq($$49, $$42))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HYBRID_HASH_JOIN [$$43, $$49][$$44, $$42]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HASH_PARTITION_EXCHANGE [$$43, $$49]  |PARTITIONED|
+                    assign [$$49] <- [$$43] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ASSIGN  |PARTITIONED|
+                      project ([$$43]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$42] <- [$$l.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ASSIGN  |PARTITIONED|
-                          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 []<-[$$43, $$o] <- tpch.Orders project ({}) [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|
-                              data-scan []<-[$$44, $$45, $$l] <- tpch.LineItem project ({l_suppkey:any}) [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|
+                              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 [$$44, $$42]  |PARTITIONED|
+                    assign [$$42] <- [$$l.getField(2)] project: [$$44, $$42] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ASSIGN  |PARTITIONED|
+                      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 []<-[$$44, $$45, $$l] <- tpch.LineItem project ({l_suppkey:any}) [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_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.08.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.08.plan
index b58b618..dd677dc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.08.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.08.plan
@@ -2,49 +2,45 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$38]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      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 [$$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 ([$$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($$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]
-                    -- HASH_PARTITION_EXCHANGE [$$43, $$56]  |PARTITIONED|
-                      assign [$$56] <- [$$43] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
-                        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 []<-[$$43, $$o] <- tpch.Orders project ({}) [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 [$$44, $$47]  |PARTITIONED|
-                      project ([$$44, $$47]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$38] <- [{"o_orderkey": $$43, "l_orderkey": $$44, "l_suppkey": $$47}] project: [$$38] [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 [$$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 ([$$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($$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]
+                  -- HASH_PARTITION_EXCHANGE [$$43, $$56]  |PARTITIONED|
+                    assign [$$56] <- [$$43] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ASSIGN  |PARTITIONED|
+                      project ([$$43]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$47] <- [$$l.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ASSIGN  |PARTITIONED|
-                          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 []<-[$$43, $$o] <- tpch.Orders project ({}) [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|
-                              data-scan []<-[$$44, $$45, $$l] <- tpch.LineItem project ({l_suppkey:any}) [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|
+                              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 [$$44, $$47]  |PARTITIONED|
+                    assign [$$47] <- [$$l.getField(2)] project: [$$44, $$47] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ASSIGN  |PARTITIONED|
+                      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 []<-[$$44, $$45, $$l] <- tpch.LineItem project ({l_suppkey:any}) [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_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.10.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.10.plan
index f99a1a2..a3ad366 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.10.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.10.plan
@@ -2,63 +2,57 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$51]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$51] <- [{"n_nationkey": $$58, "s_nationkey": $$56, "c_nationkey": $$55}] [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 [$$58(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
-          order (ASC, $$58) (ASC, $$56) (ASC, $$55) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$58(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              join (eq($$55, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HYBRID_HASH_JOIN [$$58][$$55]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  join (eq($$56, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HYBRID_HASH_JOIN [$$58][$$56]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-                      project ([$$58]) [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 []<-[$$58, $$n] <- tpch.Nation project ({}) [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 [$$56]  |PARTITIONED|
-                      project ([$$56]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        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 []<-[$$59, $$s] <- tpch.Supplier project ({s_nationkey:any}) [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 [$$55]  |PARTITIONED|
-                  project ([$$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$55] <- [$$c.getField(3)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$51] <- [{"n_nationkey": $$58, "s_nationkey": $$56, "c_nationkey": $$55}] project: [$$51] [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 [$$58(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
+        order (ASC, $$58) (ASC, $$56) (ASC, $$55) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$58(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            join (eq($$55, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HYBRID_HASH_JOIN [$$58][$$55]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                join (eq($$56, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HYBRID_HASH_JOIN [$$58][$$56]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
+                    project ([$$58]) [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 []<-[$$58, $$n] <- tpch.Nation project ({}) [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 [$$56]  |PARTITIONED|
+                    assign [$$56] <- [$$s.getField(3)] project: [$$56] [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]
+                      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 []<-[$$60, $$c] <- tpch.Customer project ({c_nationkey:any}) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          data-scan []<-[$$59, $$s] <- tpch.Supplier project ({s_nationkey:any}) [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 [$$55]  |PARTITIONED|
+                assign [$$55] <- [$$c.getField(3)] project: [$$55] [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|
+                      data-scan []<-[$$60, $$c] <- tpch.Customer project ({c_nationkey:any}) [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_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.12.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.12.plan
index f99a1a2..a3ad366 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.12.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.12.plan
@@ -2,63 +2,57 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$51]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$51] <- [{"n_nationkey": $$58, "s_nationkey": $$56, "c_nationkey": $$55}] [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 [$$58(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
-          order (ASC, $$58) (ASC, $$56) (ASC, $$55) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$58(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              join (eq($$55, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- HYBRID_HASH_JOIN [$$58][$$55]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  join (eq($$56, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HYBRID_HASH_JOIN [$$58][$$56]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-                      project ([$$58]) [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 []<-[$$58, $$n] <- tpch.Nation project ({}) [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 [$$56]  |PARTITIONED|
-                      project ([$$56]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        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 []<-[$$59, $$s] <- tpch.Supplier project ({s_nationkey:any}) [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 [$$55]  |PARTITIONED|
-                  project ([$$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$55] <- [$$c.getField(3)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$51] <- [{"n_nationkey": $$58, "s_nationkey": $$56, "c_nationkey": $$55}] project: [$$51] [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 [$$58(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
+        order (ASC, $$58) (ASC, $$56) (ASC, $$55) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$58(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            join (eq($$55, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- HYBRID_HASH_JOIN [$$58][$$55]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                join (eq($$56, $$58)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HYBRID_HASH_JOIN [$$58][$$56]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
+                    project ([$$58]) [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 []<-[$$58, $$n] <- tpch.Nation project ({}) [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 [$$56]  |PARTITIONED|
+                    assign [$$56] <- [$$s.getField(3)] project: [$$56] [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]
+                      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 []<-[$$60, $$c] <- tpch.Customer project ({c_nationkey:any}) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          data-scan []<-[$$59, $$s] <- tpch.Supplier project ({s_nationkey:any}) [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 [$$55]  |PARTITIONED|
+                assign [$$55] <- [$$c.getField(3)] project: [$$55] [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|
+                      data-scan []<-[$$60, $$c] <- tpch.Customer project ({c_nationkey:any}) [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_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.14.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.14.plan
index 7621023..6097059 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.14.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.14.plan
@@ -2,65 +2,59 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$51]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$51] <- [{"n_nationkey": $$59, "s_nationkey": $$56, "c_nationkey": $$55}] [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 [$$59(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
-          order (ASC, $$59) (ASC, $$56) (ASC, $$55) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$59(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$59, $$56, $$55]) [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|
-                  join (and(eq($$55, $$59), eq($$56, $$66))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HYBRID_HASH_JOIN [$$59, $$56][$$55, $$66]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HASH_PARTITION_EXCHANGE [$$59, $$56]  |PARTITIONED|
-                      project ([$$56, $$59]) [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 [$$59, $$n] <- index-search("Nation", 0, "Default", "tpch", "Nation", true, true, 1, $$56, 1, $$56, true, true, true) project ({}) [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, $$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 [$$56]  |PARTITIONED|
-                                  project ([$$56]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$51] <- [{"n_nationkey": $$59, "s_nationkey": $$56, "c_nationkey": $$55}] project: [$$51] [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 [$$59(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
+        order (ASC, $$59) (ASC, $$56) (ASC, $$55) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$59(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            project ([$$59, $$56, $$55]) [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|
+                join (and(eq($$55, $$59), eq($$56, $$66))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HYBRID_HASH_JOIN [$$59, $$56][$$55, $$66]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HASH_PARTITION_EXCHANGE [$$59, $$56]  |PARTITIONED|
+                    project ([$$56, $$59]) [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 [$$59, $$n] <- index-search("Nation", 0, "Default", "tpch", "Nation", true, true, 1, $$56, 1, $$56, true, true, true) project ({}) [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, $$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 [$$56]  |PARTITIONED|
+                                assign [$$56] <- [$$s.getField(3)] project: [$$56] [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|
-                                    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 []<-[$$58, $$s] <- tpch.Supplier project ({s_nationkey:any}) [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|
-                                          data-scan []<-[$$58, $$s] <- tpch.Supplier project ({s_nationkey:any}) [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 [$$55, $$66]  |PARTITIONED|
-                      assign [$$66] <- [$$55] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          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 [$$55, $$66]  |PARTITIONED|
+                    assign [$$66] <- [$$55] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ASSIGN  |PARTITIONED|
+                      assign [$$55] <- [$$c.getField(3)] project: [$$55] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- ASSIGN  |PARTITIONED|
-                        project ([$$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        project ([$$c]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$55] <- [$$c.getField(3)] [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|
+                            data-scan []<-[$$60, $$c] <- tpch.Customer project ({c_nationkey:any}) [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|
-                                data-scan []<-[$$60, $$c] <- tpch.Customer project ({c_nationkey:any}) [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|
+                                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/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.16.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.16.plan
index 7621023..6097059 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.16.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.16.plan
@@ -2,65 +2,59 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$51]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$51] <- [{"n_nationkey": $$59, "s_nationkey": $$56, "c_nationkey": $$55}] [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 [$$59(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
-          order (ASC, $$59) (ASC, $$56) (ASC, $$55) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$59(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$59, $$56, $$55]) [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|
-                  join (and(eq($$55, $$59), eq($$56, $$66))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HYBRID_HASH_JOIN [$$59, $$56][$$55, $$66]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- HASH_PARTITION_EXCHANGE [$$59, $$56]  |PARTITIONED|
-                      project ([$$56, $$59]) [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 [$$59, $$n] <- index-search("Nation", 0, "Default", "tpch", "Nation", true, true, 1, $$56, 1, $$56, true, true, true) project ({}) [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, $$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 [$$56]  |PARTITIONED|
-                                  project ([$$56]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$51] <- [{"n_nationkey": $$59, "s_nationkey": $$56, "c_nationkey": $$55}] project: [$$51] [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 [$$59(ASC), $$56(ASC), $$55(ASC) ]  |PARTITIONED|
+        order (ASC, $$59) (ASC, $$56) (ASC, $$55) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$59(ASC), $$56(ASC), $$55(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            project ([$$59, $$56, $$55]) [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|
+                join (and(eq($$55, $$59), eq($$56, $$66))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HYBRID_HASH_JOIN [$$59, $$56][$$55, $$66]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- HASH_PARTITION_EXCHANGE [$$59, $$56]  |PARTITIONED|
+                    project ([$$56, $$59]) [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 [$$59, $$n] <- index-search("Nation", 0, "Default", "tpch", "Nation", true, true, 1, $$56, 1, $$56, true, true, true) project ({}) [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, $$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 [$$56]  |PARTITIONED|
+                                assign [$$56] <- [$$s.getField(3)] project: [$$56] [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|
-                                    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 []<-[$$58, $$s] <- tpch.Supplier project ({s_nationkey:any}) [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|
-                                          data-scan []<-[$$58, $$s] <- tpch.Supplier project ({s_nationkey:any}) [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 [$$55, $$66]  |PARTITIONED|
-                      assign [$$66] <- [$$55] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          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 [$$55, $$66]  |PARTITIONED|
+                    assign [$$66] <- [$$55] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- ASSIGN  |PARTITIONED|
+                      assign [$$55] <- [$$c.getField(3)] project: [$$55] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                       -- ASSIGN  |PARTITIONED|
-                        project ([$$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        project ([$$c]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$55] <- [$$c.getField(3)] [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|
+                            data-scan []<-[$$60, $$c] <- tpch.Customer project ({c_nationkey:any}) [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|
-                                data-scan []<-[$$60, $$c] <- tpch.Customer project ({c_nationkey:any}) [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|
+                                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/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.007.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.007.plan
index b98f453..6fa2bb0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.007.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.007.plan
@@ -2,61 +2,53 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$52] <- [{"t1_id": $$53, "t2_id": $$54}] [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 [$$53(ASC), $$54(ASC) ]  |PARTITIONED|
-          order (ASC, $$53) (ASC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$53, $$54]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select ($$62) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  window-aggregate [$$62] <- [win-mark-first-missing-impl($$54)] partition [$$53] order (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- WINDOW_STREAM  |PARTITIONED|
+    assign [$$52] <- [{"t1_id": $$53, "t2_id": $$54}] project: [$$52] [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 [$$53(ASC), $$54(ASC) ]  |PARTITIONED|
+        order (ASC, $$53) (ASC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select ($$62) project: [$$53, $$54] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              window-aggregate [$$62] <- [win-mark-first-missing-impl($$54)] partition [$$53] order (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- WINDOW_STREAM  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$53) (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STABLE_SORT [$$53(ASC), $$54(DESC)]  |PARTITIONED|
                     exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      order (ASC, $$53) (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STABLE_SORT [$$53(ASC), $$54(DESC)]  |PARTITIONED|
-                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
-                          project ([$$53, $$54]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            select (eq($$55, $$t2.getField("c_int32"))) retain-untrue ($$54 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_SELECT  |PARTITIONED|
-                              project ([$$53, $$55, $$54, $$t2]) [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-unnest-map [$$54, $$t2] <- index-search("ds_inner_untyped", 0, "Default", "test", "ds_inner_untyped", true, false, 1, $$61, 1, $$61, true, true, true) project ({c_int32:any}) [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, $$61) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
-                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          project ([$$53, $$55, $$61]) [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-unnest-map [$$60, $$61] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_untyped", true, true, 1, $$55, 1, $$55, true, true, true) [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]
-                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                  project ([$$53, $$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    assign [$$55] <- [int32($$t1.getField("c_int32"))] [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]
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped project ({c_int32:any}) [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|
+                    -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                      select (eq($$55, $$t2.getField("c_int32"))) retain-untrue ($$54 <- missing) project: [$$53, $$54] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- STREAM_SELECT  |PARTITIONED|
+                        project ([$$53, $$55, $$54, $$t2]) [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-unnest-map [$$54, $$t2] <- index-search("ds_inner_untyped", 0, "Default", "test", "ds_inner_untyped", true, false, 1, $$61, 1, $$61, true, true, true) project ({c_int32:any}) [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, $$61) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
+                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    project ([$$53, $$55, $$61]) [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-unnest-map [$$60, $$61] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_untyped", true, true, 1, $$55, 1, $$55, true, true, true) [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]
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            assign [$$55] <- [int32($$t1.getField("c_int32"))] project: [$$53, $$55] [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]
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped project ({c_int32:any}) [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_column/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.008.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.008.plan
index 625d7fb..600fbff 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.008.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.008.plan
@@ -2,73 +2,37 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$52] <- [{"t1_id": $$73, "t2_id": $$54}] [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 [$$73(ASC), $$54(ASC) ]  |PARTITIONED|
-          order (ASC, $$73) (ASC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$73(ASC), $$54(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$73, $$54]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select ($$74) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  window-aggregate [$$74] <- [win-mark-first-missing-impl($$54)] partition [$$73] order (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- WINDOW_STREAM  |PARTITIONED|
+    assign [$$52] <- [{"t1_id": $$73, "t2_id": $$54}] project: [$$52] [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 [$$73(ASC), $$54(ASC) ]  |PARTITIONED|
+        order (ASC, $$73) (ASC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$73(ASC), $$54(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select ($$74) project: [$$73, $$54] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              window-aggregate [$$74] <- [win-mark-first-missing-impl($$54)] partition [$$73] order (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- WINDOW_STREAM  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$73) (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STABLE_SORT [$$73(ASC), $$54(DESC)]  |PARTITIONED|
                     exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      order (ASC, $$73) (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STABLE_SORT [$$73(ASC), $$54(DESC)]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$73]  |PARTITIONED|
+                      union ($$70, $$61, $$54) ($$53, $$53, $$73) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- UNION_ALL  |PARTITIONED|
                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HASH_PARTITION_EXCHANGE [$$73]  |PARTITIONED|
-                          union ($$70, $$61, $$54) ($$53, $$53, $$73) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- UNION_ALL  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          select (eq($$55, $$71.getField(1))) retain-untrue ($$70 <- missing) project: [$$70, $$53] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- STREAM_SELECT  |PARTITIONED|
                             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              project ([$$70, $$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                select (eq($$55, $$71.getField(1))) retain-untrue ($$70 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    left-outer-unnest-map [$$70, $$71] <- index-search("ds_inner_typed", 0, "Default", "test", "ds_inner_typed", true, false, 1, $$61, 1, $$61, true, true, true) project ({c_int32:any}) [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|
-                                        project ([$$53, $$55, $$61]) [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|
-                                            split ($$62) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- SPLIT  |PARTITIONED|
-                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                left-outer-unnest-map [$$60, $$61, $$62] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_typed", true, true, 1, $$55, 1, $$55, true, true, true) [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]
-                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                    project ([$$53, $$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      assign [$$55] <- [int32($$t1.getField("c_int32"))] [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]
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped project ({c_int32:any}) [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  |PARTITIONED|
-                              project ([$$61, $$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                select (eq($$55, $$60)) retain-untrue ($$61 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  project ([$$53, $$55, $$60, $$61]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              left-outer-unnest-map [$$70, $$71] <- index-search("ds_inner_typed", 0, "Default", "test", "ds_inner_typed", true, false, 1, $$61, 1, $$61, true, true, true) project ({c_int32:any}) [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|
+                                  project ([$$53, $$55, $$61]) [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|
@@ -80,15 +44,39 @@
                                           -- BTREE_SEARCH  |PARTITIONED|
                                             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                             -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                              project ([$$53, $$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                assign [$$55] <- [int32($$t1.getField("c_int32"))] [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]
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped project ({c_int32:any}) [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|
+                                              assign [$$55] <- [int32($$t1.getField("c_int32"))] project: [$$53, $$55] [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]
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped project ({c_int32:any}) [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  |PARTITIONED|
+                          select (eq($$55, $$60)) retain-untrue ($$61 <- missing) project: [$$61, $$53] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- STREAM_SELECT  |PARTITIONED|
+                            project ([$$53, $$55, $$60, $$61]) [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|
+                                split ($$62) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- SPLIT  |PARTITIONED|
+                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    left-outer-unnest-map [$$60, $$61, $$62] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_typed", true, true, 1, $$55, 1, $$55, true, true, true) [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]
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        assign [$$55] <- [int32($$t1.getField("c_int32"))] project: [$$53, $$55] [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]
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped project ({c_int32:any}) [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_column/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.009.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.009.plan
index 07d1780..edcf4e1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.009.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.009.plan
@@ -2,61 +2,53 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$52] <- [{"t1_id": $$53, "t2_id": $$54}] [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 [$$53(ASC), $$54(ASC) ]  |PARTITIONED|
-          order (ASC, $$53) (ASC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$53, $$54]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select ($$62) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  window-aggregate [$$62] <- [win-mark-first-missing-impl($$54)] partition [$$53] order (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- WINDOW_STREAM  |PARTITIONED|
+    assign [$$52] <- [{"t1_id": $$53, "t2_id": $$54}] project: [$$52] [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 [$$53(ASC), $$54(ASC) ]  |PARTITIONED|
+        order (ASC, $$53) (ASC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select ($$62) project: [$$53, $$54] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              window-aggregate [$$62] <- [win-mark-first-missing-impl($$54)] partition [$$53] order (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- WINDOW_STREAM  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$53) (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STABLE_SORT [$$53(ASC), $$54(DESC)]  |PARTITIONED|
                     exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      order (ASC, $$53) (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STABLE_SORT [$$53(ASC), $$54(DESC)]  |PARTITIONED|
-                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
-                          project ([$$53, $$54]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            select (eq($$55, $$t2.getField("c_int32"))) retain-untrue ($$54 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_SELECT  |PARTITIONED|
-                              project ([$$53, $$55, $$54, $$t2]) [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-unnest-map [$$54, $$t2] <- index-search("ds_inner_untyped", 0, "Default", "test", "ds_inner_untyped", true, false, 1, $$61, 1, $$61, true, true, true) project ({c_int32:any}) [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, $$61) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
-                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          project ([$$53, $$55, $$61]) [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-unnest-map [$$60, $$61] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_untyped", true, true, 1, $$55, 1, $$55, true, true, true) [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]
-                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                  project ([$$53, $$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    assign [$$55] <- [int32($$t1.getField(1))] [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]
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        data-scan []<-[$$53, $$t1] <- test.ds_outer_typed project ({c_int32:any}) [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|
+                    -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                      select (eq($$55, $$t2.getField("c_int32"))) retain-untrue ($$54 <- missing) project: [$$53, $$54] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- STREAM_SELECT  |PARTITIONED|
+                        project ([$$53, $$55, $$54, $$t2]) [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-unnest-map [$$54, $$t2] <- index-search("ds_inner_untyped", 0, "Default", "test", "ds_inner_untyped", true, false, 1, $$61, 1, $$61, true, true, true) project ({c_int32:any}) [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, $$61) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
+                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    project ([$$53, $$55, $$61]) [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-unnest-map [$$60, $$61] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_untyped", true, true, 1, $$55, 1, $$55, true, true, true) [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]
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            assign [$$55] <- [int32($$t1.getField(1))] project: [$$53, $$55] [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]
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                data-scan []<-[$$53, $$t1] <- test.ds_outer_typed project ({c_int32:any}) [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_column/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.010.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.010.plan
index 704dae5..76cd017 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.010.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.010.plan
@@ -2,73 +2,37 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$52] <- [{"t1_id": $$73, "t2_id": $$54}] [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 [$$73(ASC), $$54(ASC) ]  |PARTITIONED|
-          order (ASC, $$73) (ASC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$73(ASC), $$54(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$73, $$54]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select ($$74) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  window-aggregate [$$74] <- [win-mark-first-missing-impl($$54)] partition [$$73] order (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- WINDOW_STREAM  |PARTITIONED|
+    assign [$$52] <- [{"t1_id": $$73, "t2_id": $$54}] project: [$$52] [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 [$$73(ASC), $$54(ASC) ]  |PARTITIONED|
+        order (ASC, $$73) (ASC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$73(ASC), $$54(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select ($$74) project: [$$73, $$54] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              window-aggregate [$$74] <- [win-mark-first-missing-impl($$54)] partition [$$73] order (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- WINDOW_STREAM  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$73) (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STABLE_SORT [$$73(ASC), $$54(DESC)]  |PARTITIONED|
                     exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      order (ASC, $$73) (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STABLE_SORT [$$73(ASC), $$54(DESC)]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$73]  |PARTITIONED|
+                      union ($$70, $$61, $$54) ($$53, $$53, $$73) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- UNION_ALL  |PARTITIONED|
                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HASH_PARTITION_EXCHANGE [$$73]  |PARTITIONED|
-                          union ($$70, $$61, $$54) ($$53, $$53, $$73) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- UNION_ALL  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          select (eq($$55, $$71.getField(1))) retain-untrue ($$70 <- missing) project: [$$70, $$53] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- STREAM_SELECT  |PARTITIONED|
                             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              project ([$$70, $$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                select (eq($$55, $$71.getField(1))) retain-untrue ($$70 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    left-outer-unnest-map [$$70, $$71] <- index-search("ds_inner_typed", 0, "Default", "test", "ds_inner_typed", true, false, 1, $$61, 1, $$61, true, true, true) project ({c_int32:any}) [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|
-                                        project ([$$53, $$55, $$61]) [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|
-                                            split ($$62) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- SPLIT  |PARTITIONED|
-                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                left-outer-unnest-map [$$60, $$61, $$62] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_typed", true, true, 1, $$55, 1, $$55, true, true, true) [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]
-                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                    project ([$$53, $$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      assign [$$55] <- [int32($$t1.getField(1))] [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]
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          data-scan []<-[$$53, $$t1] <- test.ds_outer_typed project ({c_int32:any}) [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  |PARTITIONED|
-                              project ([$$61, $$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                select (eq($$55, $$60)) retain-untrue ($$61 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  project ([$$53, $$55, $$60, $$61]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              left-outer-unnest-map [$$70, $$71] <- index-search("ds_inner_typed", 0, "Default", "test", "ds_inner_typed", true, false, 1, $$61, 1, $$61, true, true, true) project ({c_int32:any}) [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|
+                                  project ([$$53, $$55, $$61]) [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|
@@ -80,15 +44,39 @@
                                           -- BTREE_SEARCH  |PARTITIONED|
                                             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                             -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                              project ([$$53, $$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                assign [$$55] <- [int32($$t1.getField(1))] [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]
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    data-scan []<-[$$53, $$t1] <- test.ds_outer_typed project ({c_int32:any}) [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|
+                                              assign [$$55] <- [int32($$t1.getField(1))] project: [$$53, $$55] [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]
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  data-scan []<-[$$53, $$t1] <- test.ds_outer_typed project ({c_int32:any}) [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  |PARTITIONED|
+                          select (eq($$55, $$60)) retain-untrue ($$61 <- missing) project: [$$61, $$53] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- STREAM_SELECT  |PARTITIONED|
+                            project ([$$53, $$55, $$60, $$61]) [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|
+                                split ($$62) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- SPLIT  |PARTITIONED|
+                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    left-outer-unnest-map [$$60, $$61, $$62] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_typed", true, true, 1, $$55, 1, $$55, true, true, true) [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]
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        assign [$$55] <- [int32($$t1.getField(1))] project: [$$53, $$55] [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]
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            data-scan []<-[$$53, $$t1] <- test.ds_outer_typed project ({c_int32:any}) [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_column/limit/offset_without_limit/offset_without_limit.6.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/limit/offset_without_limit/offset_without_limit.6.plan
index e167bb8..e493a7d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/limit/offset_without_limit/offset_without_limit.6.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/limit/offset_without_limit/offset_without_limit.6.plan
@@ -4,19 +4,17 @@
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit offset 98 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$16]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$16] <- [{"id": $$18, "dblpid": $$paper.getField(1)}] [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 [$$18(ASC) ]  |PARTITIONED|
-            order (ASC, $$18) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                data-scan []<-[$$18, $$paper] <- test.DBLP1 project ({dblpid:any}) [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|
+      assign [$$16] <- [{"id": $$18, "dblpid": $$paper.getField(1)}] project: [$$16] [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 [$$18(ASC) ]  |PARTITIONED|
+          order (ASC, $$18) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$18, $$paper] <- test.DBLP1 project ({dblpid:any}) [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_column/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.plan
index 2ba2146..342bf37 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.plan
@@ -6,35 +6,33 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       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 [$$22, $$23, $$c] <- index-search("LineItem", 0, "Default", "test", "LineItem", false, false, 2, $$27, $$28, 2, $$27, $$28, true, true, true) condition (lt($$c.getField(2), 150)) limit 5 project ({l_suppkey:any,l_shipdate:any}) [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, $$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 ([$$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 [$$26, $$27, $$28] <- index-search("idx_LineItem_suppkey", 0, "Default", "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 [$$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|
+        assign [$$20] <- [{"shipdate": substring($$c.getField(10), 0, 4), "suppkey": gt($$21, 0)}] project: [$$20] [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 [$$22, $$23, $$c] <- index-search("LineItem", 0, "Default", "test", "LineItem", false, false, 2, $$27, $$28, 2, $$27, $$28, true, true, true) condition (lt($$c.getField(2), 150)) limit 5 project ({l_suppkey:any,l_shipdate:any}) [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, $$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 ([$$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 [$$26, $$27, $$28] <- index-search("idx_LineItem_suppkey", 0, "Default", "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 [$$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_column/objects/load-record-fields/load-record-fields.4.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/objects/load-record-fields/load-record-fields.4.plan
index a841292..698f06d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/objects/load-record-fields/load-record-fields.4.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/objects/load-record-fields/load-record-fields.4.plan
@@ -2,25 +2,23 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$30]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$30] <- [$$md.getField("name")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$md]) [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 [$$32(ASC) ]  |PARTITIONED|
-            order (ASC, $$32) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                select (neq(uuid(), uuid())) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$32, $$md] <- test.MyDataset project ({name:any}) [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|
+    assign [$$30] <- [$$md.getField("name")] project: [$$30] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$md]) [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 [$$32(ASC) ]  |PARTITIONED|
+          order (ASC, $$32) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              select (neq(uuid(), uuid())) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$32, $$md] <- test.MyDataset project ({name:any}) [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_column/objects/load-record-fields/load-record-fields.6.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/objects/load-record-fields/load-record-fields.6.plan
index 6a76140..7117b98 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/objects/load-record-fields/load-record-fields.6.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/objects/load-record-fields/load-record-fields.6.plan
@@ -2,25 +2,23 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$30]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$30] <- [$$md.getField("name")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |PARTITIONED|
-        project ([$$md]) [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 [$$32(ASC) ]  |PARTITIONED|
-            order (ASC, $$32) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                select (neq(current-date(), date: { 1980-09-10 })) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$32, $$md] <- test.MyDataset project ({name:any}) [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|
+    assign [$$30] <- [$$md.getField("name")] project: [$$30] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      project ([$$md]) [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 [$$32(ASC) ]  |PARTITIONED|
+          order (ASC, $$32) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              select (neq(current-date(), date: { 1980-09-10 })) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STREAM_SELECT  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$32, $$md] <- test.MyDataset project ({name:any}) [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_column/subquery/query-ASTERIXDB-3538/query-ASTERIXDB-3538.3.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/subquery/query-ASTERIXDB-3538/query-ASTERIXDB-3538.3.plan
new file mode 100644
index 0000000..34d7663
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/subquery/query-ASTERIXDB-3538/query-ASTERIXDB-3538.3.plan
@@ -0,0 +1,20 @@
+distribute result [$$311] [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|
+    assign [$$311] <- [{"id": $$345}] project: [$$311] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (not(is-null($$345))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        assign [$$345] <- [string-default-null($$s.getField("id"))] project: [$$345] [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 []<-[$$314, $$s] <- test.dat1 project ({id:any}) [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_column/subquery/query-ASTERIXDB-3538/query-ASTERIXDB-3538.5.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/subquery/query-ASTERIXDB-3538/query-ASTERIXDB-3538.5.plan
new file mode 100644
index 0000000..c94e333
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/subquery/query-ASTERIXDB-3538/query-ASTERIXDB-3538.5.plan
@@ -0,0 +1,35 @@
+distribute result [$$317] [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|
+    assign [$$317] <- [{"dat3": {"id": $$351, "a": string-default-null($$345), "d": int64-default-null($$318)}}] project: [$$317] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |PARTITIONED|
+      select (le($$318, get-item($$295, 0))) project: [$$351, $$345, $$318] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_SELECT  |PARTITIONED|
+        project ([$$351, $$345, $$318, $$295]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STREAM_PROJECT  |PARTITIONED|
+          subplan {
+                    aggregate [$$295] <- [listify($$330)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- AGGREGATE  |LOCAL|
+                      aggregate [$$330] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        unnest $$333 <- scan-collection($$343) [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: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- SUBPLAN  |PARTITIONED|
+            select (not(is-null($$351))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              assign [$$351, $$345, $$318, $$343] <- [string-default-null($$s.getField("id")), $$s.getField("a"), $$s.getField("d"), $$s.getField("e")] project: [$$351, $$345, $$318, $$343] [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 []<-[$$319, $$s] <- test.dat1 project ({a:any,d:any,e:[any],id:any}) [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_column/tpch/query-plans/query-plans.03.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.03.plan
index 04cf977..49dcfab 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.03.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.03.plan
@@ -2,43 +2,39 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$134]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$134] <- [{"$1": 1, "l_returnflag": $$l_returnflag, "l_linestatus": $$l_linestatus, "sum_qty": $$140, "sum_base_price": $$141, "sum_disc_price": $$142, "sum_charge": $$143, "avg_qty": $$144, "avg_price": $$145, "avg_disc": $$146, "count_order": $$147}] [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 [$$l_returnflag(ASC), $$l_linestatus(ASC) ]  |PARTITIONED|
-          group by ([$$l_returnflag := $$160; $$l_linestatus := $$161]) decor ([]) {
-                    aggregate [$$140, $$141, $$142, $$143, $$144, $$145, $$146, $$147] <- [agg-global-sql-sum($$152), agg-global-sql-sum($$153), agg-global-sql-sum($$154), agg-global-sql-sum($$155), agg-global-sql-avg($$156), agg-global-sql-avg($$157), agg-global-sql-avg($$158), agg-sql-sum($$159)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$160, $$161]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$160, $$161]  |PARTITIONED|
-              group by ([$$160 := $$135; $$161 := $$136]) decor ([]) {
-                        aggregate [$$152, $$153, $$154, $$155, $$156, $$157, $$158, $$159] <- [agg-local-sql-sum($$88), agg-local-sql-sum($$93), agg-local-sql-sum(numeric-multiply($$93, numeric-subtract(1, $$149))), agg-local-sql-sum(numeric-multiply(numeric-multiply($$93, numeric-subtract(1, $$149)), numeric-add(1, $$151))), agg-local-sql-avg($$88), agg-local-sql-avg($$93), agg-local-sql-avg($$149), agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$135, $$136]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$88, $$93, $$149, $$151, $$135, $$136]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    select (le($$l.getField(10), "1998-09-16")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_SELECT  |PARTITIONED|
-                      assign [$$136, $$135, $$88, $$149, $$151, $$93] <- [$$l.getField(9), $$l.getField(8), $$l.getField(4), $$l.getField(6), $$l.getField(7), $$l.getField(5)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
-                        project ([$$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
+    assign [$$134] <- [{"$1": 1, "l_returnflag": $$l_returnflag, "l_linestatus": $$l_linestatus, "sum_qty": $$140, "sum_base_price": $$141, "sum_disc_price": $$142, "sum_charge": $$143, "avg_qty": $$144, "avg_price": $$145, "avg_disc": $$146, "count_order": $$147}] project: [$$134] [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 [$$l_returnflag(ASC), $$l_linestatus(ASC) ]  |PARTITIONED|
+        group by ([$$l_returnflag := $$160; $$l_linestatus := $$161]) decor ([]) {
+                  aggregate [$$140, $$141, $$142, $$143, $$144, $$145, $$146, $$147] <- [agg-global-sql-sum($$152), agg-global-sql-sum($$153), agg-global-sql-sum($$154), agg-global-sql-sum($$155), agg-global-sql-avg($$156), agg-global-sql-avg($$157), agg-global-sql-avg($$158), agg-sql-sum($$159)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$160, $$161]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$160, $$161]  |PARTITIONED|
+            group by ([$$160 := $$135; $$161 := $$136]) decor ([]) {
+                      aggregate [$$152, $$153, $$154, $$155, $$156, $$157, $$158, $$159] <- [agg-local-sql-sum($$88), agg-local-sql-sum($$93), agg-local-sql-sum(numeric-multiply($$93, numeric-subtract(1, $$149))), agg-local-sql-sum(numeric-multiply(numeric-multiply($$93, numeric-subtract(1, $$149)), numeric-add(1, $$151))), agg-local-sql-avg($$88), agg-local-sql-avg($$93), agg-local-sql-avg($$149), agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$135, $$136]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                select (le($$l.getField(10), "1998-09-16")) project: [$$88, $$93, $$149, $$151, $$135, $$136] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STREAM_SELECT  |PARTITIONED|
+                  assign [$$136, $$135, $$88, $$149, $$151, $$93] <- [$$l.getField(9), $$l.getField(8), $$l.getField(4), $$l.getField(6), $$l.getField(7), $$l.getField(5)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ASSIGN  |PARTITIONED|
+                    project ([$$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 []<-[$$138, $$139, $$l] <- tpch.lineitem project ({l_returnflag:any,l_tax:any,l_quantity:any,l_linestatus:any,l_extendedprice:any,l_shipdate:any,l_discount:any}) filter on: le($$l.getField(10), "1998-09-16") range-filter on: le($$l.getField(10), "1998-09-16") [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|
-                            data-scan []<-[$$138, $$139, $$l] <- tpch.lineitem project ({l_returnflag:any,l_tax:any,l_quantity:any,l_linestatus:any,l_extendedprice:any,l_shipdate:any,l_discount:any}) filter on: le($$l.getField(10), "1998-09-16") range-filter on: le($$l.getField(10), "1998-09-16") [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|
+                            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/tpch/query-plans/query-plans.04.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.04.plan
index 980ef8a..757b580 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.04.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.04.plan
@@ -4,288 +4,264 @@
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit 100 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$194]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$194] <- [{"$1": 2, "s_acctbal": $$237, "s_name": $$239, "n_name": $$238, "p_partkey": $$250, "p_mfgr": $$244, "s_address": $$245, "s_phone": $$246, "s_comment": $$247}] [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 [$$237(DESC), $$238(ASC), $$239(ASC), $$250(ASC) ]  |PARTITIONED|
-            limit 100 [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]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (topK: 100) (DESC, $$237) (ASC, $$238) (ASC, $$239) (ASC, $$250) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STABLE_SORT [topK: 100] [$$237(DESC), $$238(ASC), $$239(ASC), $$250(ASC)]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$237, $$239, $$238, $$250, $$244, $$245, $$246, $$247]) [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|
-                        join (eq($$221, $$206)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HYBRID_HASH_JOIN [$$221][$$206]  |PARTITIONED|
-                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- HASH_PARTITION_EXCHANGE [$$221]  |PARTITIONED|
-                            project ([$$237, $$239, $$238, $$250, $$244, $$245, $$246, $$247, $$221]) [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|
-                                join (eq($$225, $$205)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- HYBRID_HASH_JOIN [$$225][$$205]  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- HASH_PARTITION_EXCHANGE [$$225]  |PARTITIONED|
-                                    project ([$$237, $$239, $$250, $$244, $$245, $$246, $$247, $$225]) [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|
-                                        join (and(eq($$250, $$203), eq($$202, $$231))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- HYBRID_HASH_JOIN [$$250, $$231][$$203, $$202]  |PARTITIONED|
-                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            project ([$$250, $$244, $$231]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              assign [$$231] <- [get-item($$181, 0)] [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]
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  group by ([$$250 := $$200]) decor ([$$244]) {
-                                                            aggregate [$$181] <- [listify($$215)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                            -- AGGREGATE  |LOCAL|
-                                                              aggregate [$$215] <- [agg-sql-min($$208)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                              -- AGGREGATE  |LOCAL|
-                                                                select (not(is-missing($$249))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                -- STREAM_SELECT  |LOCAL|
-                                                                  nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                         } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- PRE_CLUSTERED_GROUP_BY[$$200]  |PARTITIONED|
-                                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      order (ASC, $$200) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- STABLE_SORT [$$200(ASC)]  |PARTITIONED|
-                                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          project ([$$244, $$208, $$249, $$200]) [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 (eq($$207, $$200)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                              -- HYBRID_HASH_JOIN [$$200][$$207]  |PARTITIONED|
-                                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                -- HASH_PARTITION_EXCHANGE [$$200]  |PARTITIONED|
-                                                                  project ([$$244, $$200]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    select (and(eq($$p.getField(5), 37), like($$p.getField(4), "%COPPER"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                    -- STREAM_SELECT  |PARTITIONED|
-                                                                      assign [$$244] <- [$$p.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]
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          data-scan []<-[$$200, $$p] <- tpch.part project ({p_type:any,p_mfgr:any,p_size:any}) [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 [$$207]  |PARTITIONED|
-                                                                  assign [$$249] <- [true] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                  -- ASSIGN  |PARTITIONED|
-                                                                    project ([$$208, $$207]) [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|
-                                                                        join (eq($$218, $$213)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                        -- HYBRID_HASH_JOIN [$$218][$$213]  |PARTITIONED|
-                                                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                          -- HASH_PARTITION_EXCHANGE [$$218]  |PARTITIONED|
-                                                                            project ([$$208, $$207, $$218]) [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|
-                                                                                join (eq($$207, $$209)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                -- HYBRID_HASH_JOIN [$$207][$$209]  |PARTITIONED|
-                                                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                  -- HASH_PARTITION_EXCHANGE [$$207]  |PARTITIONED|
-                                                                                    project ([$$207]) [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 []<-[$$207, $$p2] <- tpch.part project ({p_type:any,p_mfgr:any,p_size:any}) [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 [$$209]  |PARTITIONED|
-                                                                                    project ([$$208, $$218, $$209]) [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|
-                                                                                        join (eq($$223, $$212)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                        -- HYBRID_HASH_JOIN [$$223][$$212]  |PARTITIONED|
-                                                                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                          -- HASH_PARTITION_EXCHANGE [$$223]  |PARTITIONED|
-                                                                                            project ([$$208, $$209, $$223]) [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|
-                                                                                                join (eq($$211, $$210)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                -- HYBRID_HASH_JOIN [$$210][$$211]  |PARTITIONED|
-                                                                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                    replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                    -- REPLICATE  |PARTITIONED|
-                                                                                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                      -- HASH_PARTITION_EXCHANGE [$$210]  |PARTITIONED|
-                                                                                                        project ([$$208, $$209, $$210]) [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 []<-[$$208, $$209, $$210, $$ps2] <- tpch.partsupp project ({}) [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 [$$211]  |PARTITIONED|
-                                                                                                    project ([$$223, $$211]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                      assign [$$223] <- [$$s2.getField(3)] [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]
-                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          data-scan []<-[$$211, $$s2] <- tpch.supplier project ({s_comment:any,s_phone:any,s_nationkey:any,s_name:any,s_address:any,s_acctbal:any}) [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 [$$212]  |PARTITIONED|
-                                                                                            project ([$$218, $$212]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                              assign [$$218] <- [$$n2.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]
-                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                  replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                  -- REPLICATE  |PARTITIONED|
-                                                                                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                      data-scan []<-[$$212, $$n2] <- tpch.nation project ({n_regionkey:any,n_name:any}) [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  |PARTITIONED|
-                                                                            replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                            -- REPLICATE  |PARTITIONED|
-                                                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                              -- HASH_PARTITION_EXCHANGE [$$213]  |PARTITIONED|
-                                                                                project ([$$213]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                  select (eq($$r2.getField(1), "EUROPE")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                  -- STREAM_SELECT  |PARTITIONED|
-                                                                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      data-scan []<-[$$213, $$r2] <- tpch.region project ({r_name:any}) filter on: eq($$r2.getField(1), "EUROPE") range-filter on: eq($$r2.getField(1), "EUROPE") [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 [$$203]  |PARTITIONED|
-                                            project ([$$237, $$239, $$245, $$246, $$247, $$225, $$203, $$202]) [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|
-                                                join (eq($$201, $$204)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- HYBRID_HASH_JOIN [$$201][$$204]  |PARTITIONED|
-                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- HASH_PARTITION_EXCHANGE [$$201]  |PARTITIONED|
-                                                    project ([$$237, $$239, $$245, $$246, $$247, $$225, $$201]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      assign [$$247, $$246, $$245, $$237, $$239, $$225] <- [$$s.getField(6), $$s.getField(4), $$s.getField(2), $$s.getField(5), $$s.getField(1), $$s.getField(3)] [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]
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          data-scan []<-[$$201, $$s] <- tpch.supplier project ({s_comment:any,s_phone:any,s_nationkey:any,s_name:any,s_address:any,s_acctbal:any}) [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  |PARTITIONED|
-                                                    project ([$$202, $$203, $$204]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      assign [$$202, $$203, $$204] <- [$$208, $$209, $$210] [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]
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                          -- REPLICATE  |PARTITIONED|
-                                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                            -- HASH_PARTITION_EXCHANGE [$$210]  |PARTITIONED|
-                                                              project ([$$208, $$209, $$210]) [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 []<-[$$208, $$209, $$210, $$ps2] <- tpch.partsupp project ({}) [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 [$$205]  |PARTITIONED|
-                                    project ([$$238, $$221, $$205]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      assign [$$238, $$221] <- [$$n.getField(1), $$n.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- ASSIGN  |PARTITIONED|
-                                        project ([$$205, $$n]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          assign [$$205, $$n] <- [$$212, $$n2] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      assign [$$194] <- [{"$1": 2, "s_acctbal": $$237, "s_name": $$239, "n_name": $$238, "p_partkey": $$250, "p_mfgr": $$244, "s_address": $$245, "s_phone": $$246, "s_comment": $$247}] project: [$$194] [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 [$$237(DESC), $$238(ASC), $$239(ASC), $$250(ASC) ]  |PARTITIONED|
+          limit 100 [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]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              order (topK: 100) (DESC, $$237) (ASC, $$238) (ASC, $$239) (ASC, $$250) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STABLE_SORT [topK: 100] [$$237(DESC), $$238(ASC), $$239(ASC), $$250(ASC)]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  project ([$$237, $$239, $$238, $$250, $$244, $$245, $$246, $$247]) [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|
+                      join (eq($$221, $$206)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- HYBRID_HASH_JOIN [$$221][$$206]  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- HASH_PARTITION_EXCHANGE [$$221]  |PARTITIONED|
+                          project ([$$237, $$239, $$238, $$250, $$244, $$245, $$246, $$247, $$221]) [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|
+                              join (eq($$225, $$205)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- HYBRID_HASH_JOIN [$$225][$$205]  |PARTITIONED|
+                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- HASH_PARTITION_EXCHANGE [$$225]  |PARTITIONED|
+                                  project ([$$237, $$239, $$250, $$244, $$245, $$246, $$247, $$225]) [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|
+                                      join (and(eq($$250, $$203), eq($$202, $$231))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- HYBRID_HASH_JOIN [$$250, $$231][$$203, $$202]  |PARTITIONED|
+                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          assign [$$231] <- [get-item($$181, 0)] project: [$$250, $$244, $$231] [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]
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- REPLICATE  |PARTITIONED|
+                                              group by ([$$250 := $$200]) decor ([$$244]) {
+                                                        aggregate [$$181] <- [listify($$215)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                        -- AGGREGATE  |LOCAL|
+                                                          aggregate [$$215] <- [agg-sql-min($$208)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                          -- AGGREGATE  |LOCAL|
+                                                            select (not(is-missing($$249))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                            -- STREAM_SELECT  |LOCAL|
+                                                              nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- PRE_CLUSTERED_GROUP_BY[$$200]  |PARTITIONED|
                                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  data-scan []<-[$$212, $$n2] <- tpch.nation project ({n_regionkey:any,n_name:any}) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                                  order (ASC, $$200) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                  -- STABLE_SORT [$$200(ASC)]  |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  |PARTITIONED|
-                            project ([$$206]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              assign [$$206] <- [$$213] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- ASSIGN  |PARTITIONED|
+                                                      project ([$$244, $$208, $$249, $$200]) [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 (eq($$207, $$200)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                          -- HYBRID_HASH_JOIN [$$200][$$207]  |PARTITIONED|
+                                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                            -- HASH_PARTITION_EXCHANGE [$$200]  |PARTITIONED|
+                                                              select (and(eq($$p.getField(5), 37), like($$p.getField(4), "%COPPER"))) project: [$$244, $$200] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                assign [$$244] <- [$$p.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]
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    data-scan []<-[$$200, $$p] <- tpch.part project ({p_type:any,p_mfgr:any,p_size:any}) [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 [$$207]  |PARTITIONED|
+                                                              assign [$$249] <- [true] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                project ([$$208, $$207]) [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|
+                                                                    join (eq($$218, $$213)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                    -- HYBRID_HASH_JOIN [$$218][$$213]  |PARTITIONED|
+                                                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                      -- HASH_PARTITION_EXCHANGE [$$218]  |PARTITIONED|
+                                                                        project ([$$208, $$207, $$218]) [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|
+                                                                            join (eq($$207, $$209)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                            -- HYBRID_HASH_JOIN [$$207][$$209]  |PARTITIONED|
+                                                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                              -- HASH_PARTITION_EXCHANGE [$$207]  |PARTITIONED|
+                                                                                project ([$$207]) [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 []<-[$$207, $$p2] <- tpch.part project ({p_type:any,p_mfgr:any,p_size:any}) [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 [$$209]  |PARTITIONED|
+                                                                                project ([$$208, $$218, $$209]) [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|
+                                                                                    join (eq($$223, $$212)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                    -- HYBRID_HASH_JOIN [$$223][$$212]  |PARTITIONED|
+                                                                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                      -- HASH_PARTITION_EXCHANGE [$$223]  |PARTITIONED|
+                                                                                        project ([$$208, $$209, $$223]) [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|
+                                                                                            join (eq($$211, $$210)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                            -- HYBRID_HASH_JOIN [$$210][$$211]  |PARTITIONED|
+                                                                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                                  -- HASH_PARTITION_EXCHANGE [$$210]  |PARTITIONED|
+                                                                                                    project ([$$208, $$209, $$210]) [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 []<-[$$208, $$209, $$210, $$ps2] <- tpch.partsupp project ({}) [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 [$$211]  |PARTITIONED|
+                                                                                                assign [$$223] <- [$$s2.getField(3)] project: [$$223, $$211] [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]
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    data-scan []<-[$$211, $$s2] <- tpch.supplier project ({s_comment:any,s_phone:any,s_nationkey:any,s_name:any,s_address:any,s_acctbal:any}) [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 [$$212]  |PARTITIONED|
+                                                                                        assign [$$218] <- [$$n2.getField(2)] project: [$$218, $$212] [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]
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                data-scan []<-[$$212, $$n2] <- tpch.nation project ({n_regionkey:any,n_name:any}) [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  |PARTITIONED|
+                                                                        replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                        -- REPLICATE  |PARTITIONED|
+                                                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                          -- HASH_PARTITION_EXCHANGE [$$213]  |PARTITIONED|
+                                                                            select (eq($$r2.getField(1), "EUROPE")) project: [$$213] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                data-scan []<-[$$213, $$r2] <- tpch.region project ({r_name:any}) filter on: eq($$r2.getField(1), "EUROPE") range-filter on: eq($$r2.getField(1), "EUROPE") [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 [$$203]  |PARTITIONED|
+                                          project ([$$237, $$239, $$245, $$246, $$247, $$225, $$203, $$202]) [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|
+                                              join (eq($$201, $$204)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- HYBRID_HASH_JOIN [$$201][$$204]  |PARTITIONED|
+                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                -- HASH_PARTITION_EXCHANGE [$$201]  |PARTITIONED|
+                                                  assign [$$247, $$246, $$245, $$237, $$239, $$225] <- [$$s.getField(6), $$s.getField(4), $$s.getField(2), $$s.getField(5), $$s.getField(1), $$s.getField(3)] project: [$$237, $$239, $$245, $$246, $$247, $$225, $$201] [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]
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      data-scan []<-[$$201, $$s] <- tpch.supplier project ({s_comment:any,s_phone:any,s_nationkey:any,s_name:any,s_address:any,s_acctbal:any}) [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  |PARTITIONED|
+                                                  assign [$$202, $$203, $$204] <- [$$208, $$209, $$210] project: [$$202, $$203, $$204] [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]
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                        -- HASH_PARTITION_EXCHANGE [$$210]  |PARTITIONED|
+                                                          project ([$$208, $$209, $$210]) [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 []<-[$$208, $$209, $$210, $$ps2] <- tpch.partsupp project ({}) [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  |PARTITIONED|
-                                  replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- REPLICATE  |PARTITIONED|
-                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- HASH_PARTITION_EXCHANGE [$$213]  |PARTITIONED|
-                                      project ([$$213]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        select (eq($$r2.getField(1), "EUROPE")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- STREAM_SELECT  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
+                                  assign [$$238, $$221] <- [$$n.getField(1), $$n.getField(2)] project: [$$238, $$221, $$205] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- ASSIGN  |PARTITIONED|
+                                    assign [$$205, $$n] <- [$$212, $$n2] project: [$$205, $$n] [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]
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- REPLICATE  |PARTITIONED|
                                           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            data-scan []<-[$$213, $$r2] <- tpch.region project ({r_name:any}) filter on: eq($$r2.getField(1), "EUROPE") range-filter on: eq($$r2.getField(1), "EUROPE") [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                            data-scan []<-[$$212, $$n2] <- tpch.nation project ({n_regionkey:any,n_name:any}) [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  |PARTITIONED|
+                          assign [$$206] <- [$$213] project: [$$206] [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]
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- REPLICATE  |PARTITIONED|
+                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- HASH_PARTITION_EXCHANGE [$$213]  |PARTITIONED|
+                                  select (eq($$r2.getField(1), "EUROPE")) project: [$$213] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      data-scan []<-[$$213, $$r2] <- tpch.region project ({r_name:any}) filter on: eq($$r2.getField(1), "EUROPE") range-filter on: eq($$r2.getField(1), "EUROPE") [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_column/tpch/query-plans/query-plans.05.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.05.plan
index d889d64..9612d1f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.05.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.05.plan
@@ -4,97 +4,89 @@
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$122]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$122] <- [{"$1": 3, "l_orderkey": $$l_orderkey, "revenue": $$134, "o_orderdate": $$o_orderdate, "o_shippriority": $$o_shippriority}] [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 [$$134(DESC), $$o_orderdate(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]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (topK: 10) (DESC, $$134) (ASC, $$o_orderdate) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STABLE_SORT [topK: 10] [$$134(DESC), $$o_orderdate(ASC)]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    group by ([$$l_orderkey := $$142; $$o_orderdate := $$143; $$o_shippriority := $$144]) decor ([]) {
-                              aggregate [$$134] <- [agg-global-sql-sum($$141)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- AGGREGATE  |LOCAL|
-                                nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                           } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- SORT_GROUP_BY[$$142, $$143, $$144]  |PARTITIONED|
-                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- HASH_PARTITION_EXCHANGE [$$142, $$143, $$144]  |PARTITIONED|
-                        group by ([$$142 := $$131; $$143 := $$127; $$144 := $$125]) decor ([]) {
-                                  aggregate [$$141] <- [agg-local-sql-sum(numeric-multiply($$139, numeric-subtract(1, $$140)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- AGGREGATE  |LOCAL|
-                                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- SORT_GROUP_BY[$$131, $$127, $$125]  |PARTITIONED|
-                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            project ([$$139, $$140, $$131, $$127, $$125]) [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|
-                                join (eq($$131, $$129)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- HYBRID_HASH_JOIN [$$129][$$131]  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- HASH_PARTITION_EXCHANGE [$$129]  |PARTITIONED|
-                                    project ([$$127, $$125, $$129]) [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|
-                                        join (eq($$128, $$136)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- HYBRID_HASH_JOIN [$$128][$$136]  |PARTITIONED|
-                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- HASH_PARTITION_EXCHANGE [$$128]  |PARTITIONED|
-                                            project ([$$128]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              select (eq($$c.getField(6), "BUILDING")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  data-scan []<-[$$128, $$c] <- tpch.customer project ({c_mktsegment:any}) filter on: eq($$c.getField(6), "BUILDING") range-filter on: eq($$c.getField(6), "BUILDING") [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 [$$136]  |PARTITIONED|
-                                            select (lt($$127, "1995-03-22")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- STREAM_SELECT  |PARTITIONED|
-                                              project ([$$129, $$125, $$136, $$127]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                assign [$$125, $$136, $$127] <- [$$o.getField(7), $$o.getField(1), $$o.getField(4)] [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]
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    data-scan []<-[$$129, $$o] <- tpch.orders project ({o_orderdate:any,o_shippriority:any,o_custkey:any}) filter on: lt($$o.getField(4), "1995-03-22") range-filter on: lt($$o.getField(4), "1995-03-22") [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 [$$131]  |PARTITIONED|
-                                    project ([$$139, $$140, $$131]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      select (gt($$l.getField(10), "1995-03-22")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- STREAM_SELECT  |PARTITIONED|
-                                        assign [$$140, $$139] <- [$$l.getField(6), $$l.getField(5)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- ASSIGN  |PARTITIONED|
-                                          project ([$$131, $$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- STREAM_PROJECT  |PARTITIONED|
+      assign [$$122] <- [{"$1": 3, "l_orderkey": $$l_orderkey, "revenue": $$134, "o_orderdate": $$o_orderdate, "o_shippriority": $$o_shippriority}] project: [$$122] [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 [$$134(DESC), $$o_orderdate(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]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              order (topK: 10) (DESC, $$134) (ASC, $$o_orderdate) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STABLE_SORT [topK: 10] [$$134(DESC), $$o_orderdate(ASC)]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  group by ([$$l_orderkey := $$142; $$o_orderdate := $$143; $$o_shippriority := $$144]) decor ([]) {
+                            aggregate [$$134] <- [agg-global-sql-sum($$141)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- AGGREGATE  |LOCAL|
+                              nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                         } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- SORT_GROUP_BY[$$142, $$143, $$144]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$142, $$143, $$144]  |PARTITIONED|
+                      group by ([$$142 := $$131; $$143 := $$127; $$144 := $$125]) decor ([]) {
+                                aggregate [$$141] <- [agg-local-sql-sum(numeric-multiply($$139, numeric-subtract(1, $$140)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- AGGREGATE  |LOCAL|
+                                  nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                             } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- SORT_GROUP_BY[$$131, $$127, $$125]  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          project ([$$139, $$140, $$131, $$127, $$125]) [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|
+                              join (eq($$131, $$129)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- HYBRID_HASH_JOIN [$$129][$$131]  |PARTITIONED|
+                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- HASH_PARTITION_EXCHANGE [$$129]  |PARTITIONED|
+                                  project ([$$127, $$125, $$129]) [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|
+                                      join (eq($$128, $$136)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- HYBRID_HASH_JOIN [$$128][$$136]  |PARTITIONED|
+                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- HASH_PARTITION_EXCHANGE [$$128]  |PARTITIONED|
+                                          select (eq($$c.getField(6), "BUILDING")) project: [$$128] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- STREAM_SELECT  |PARTITIONED|
                                             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              data-scan []<-[$$130, $$131, $$l] <- tpch.lineitem project ({l_extendedprice:any,l_shipdate:any,l_discount:any}) filter on: gt($$l.getField(10), "1995-03-22") range-filter on: gt($$l.getField(10), "1995-03-22") [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              data-scan []<-[$$128, $$c] <- tpch.customer project ({c_mktsegment:any}) filter on: eq($$c.getField(6), "BUILDING") range-filter on: eq($$c.getField(6), "BUILDING") [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 [$$136]  |PARTITIONED|
+                                          select (lt($$127, "1995-03-22")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            assign [$$125, $$136, $$127] <- [$$o.getField(7), $$o.getField(1), $$o.getField(4)] project: [$$129, $$125, $$136, $$127] [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]
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                data-scan []<-[$$129, $$o] <- tpch.orders project ({o_orderdate:any,o_shippriority:any,o_custkey:any}) filter on: lt($$o.getField(4), "1995-03-22") range-filter on: lt($$o.getField(4), "1995-03-22") [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 [$$131]  |PARTITIONED|
+                                  select (gt($$l.getField(10), "1995-03-22")) project: [$$139, $$140, $$131] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    assign [$$140, $$139] <- [$$l.getField(6), $$l.getField(5)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- ASSIGN  |PARTITIONED|
+                                      project ([$$131, $$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 []<-[$$130, $$131, $$l] <- tpch.lineitem project ({l_extendedprice:any,l_shipdate:any,l_discount:any}) filter on: gt($$l.getField(10), "1995-03-22") range-filter on: gt($$l.getField(10), "1995-03-22") [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_column/tpch/query-plans/query-plans.06.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.06.plan
index 16fbd06..42ea92f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.06.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.06.plan
@@ -2,99 +2,91 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$74]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$74] <- [{"$1": 4, "o_orderpriority": $$o_orderpriority, "order_count": $$81}] [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 [$$o_orderpriority(ASC) ]  |PARTITIONED|
-          group by ([$$o_orderpriority := $$91]) decor ([]) {
-                    aggregate [$$81] <- [agg-sql-sum($$90)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$91]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$91]  |PARTITIONED|
-              group by ([$$91 := $$75]) decor ([]) {
-                        aggregate [$$90] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$75]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$75]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$74] <- [{"$1": 4, "o_orderpriority": $$o_orderpriority, "order_count": $$81}] project: [$$74] [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 [$$o_orderpriority(ASC) ]  |PARTITIONED|
+        group by ([$$o_orderpriority := $$91]) decor ([]) {
+                  aggregate [$$81] <- [agg-sql-sum($$90)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$91]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$91]  |PARTITIONED|
+            group by ([$$91 := $$75]) decor ([]) {
+                      aggregate [$$90] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$75]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                select (neq($$80, 0)) project: [$$75] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STREAM_SELECT  |PARTITIONED|
+                  project ([$$80, $$75]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- STREAM_PROJECT  |PARTITIONED|
-                    select (neq($$80, 0)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_SELECT  |PARTITIONED|
-                      project ([$$80, $$75]) [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|
+                      group by ([$$87 := $$89]) decor ([$$75]) {
+                                aggregate [$$80] <- [agg-sum($$88)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- AGGREGATE  |LOCAL|
+                                  nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                             } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- PRE_CLUSTERED_GROUP_BY[$$89]  |PARTITIONED|
                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          group by ([$$87 := $$89]) decor ([$$75]) {
-                                    aggregate [$$80] <- [agg-sum($$88)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          group by ([$$89 := $$77]) decor ([$$75]) {
+                                    aggregate [$$88] <- [agg-count({"l": $$l})] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                     -- AGGREGATE  |LOCAL|
-                                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      select (not(is-missing($$86))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- STREAM_SELECT  |LOCAL|
+                                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
                                  } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- PRE_CLUSTERED_GROUP_BY[$$89]  |PARTITIONED|
+                          -- PRE_CLUSTERED_GROUP_BY[$$77]  |PARTITIONED|
                             exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              group by ([$$89 := $$77]) decor ([$$75]) {
-                                        aggregate [$$88] <- [agg-count({"l": $$l})] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- AGGREGATE  |LOCAL|
-                                          select (not(is-missing($$86))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- STREAM_SELECT  |LOCAL|
-                                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- PRE_CLUSTERED_GROUP_BY[$$77]  |PARTITIONED|
+                              order (ASC, $$77) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- STABLE_SORT [$$77(ASC)]  |PARTITIONED|
                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  order (ASC, $$77) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- STABLE_SORT [$$77(ASC)]  |PARTITIONED|
+                                  project ([$$75, $$l, $$86, $$77]) [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|
-                                      project ([$$75, $$l, $$86, $$77]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                      left outer join (eq($$79, $$77)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- HYBRID_HASH_JOIN [$$77][$$79]  |PARTITIONED|
                                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          left outer join (eq($$79, $$77)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- HYBRID_HASH_JOIN [$$77][$$79]  |PARTITIONED|
-                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- HASH_PARTITION_EXCHANGE [$$77]  |PARTITIONED|
-                                              project ([$$75, $$77]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- HASH_PARTITION_EXCHANGE [$$77]  |PARTITIONED|
+                                          select (and(ge($$76, "1996-05-01"), lt($$76, "1996-08-01"))) project: [$$75, $$77] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            assign [$$75, $$76] <- [$$o.getField(5), $$o.getField(4)] project: [$$77, $$75, $$76] [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]
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                data-scan []<-[$$77, $$o] <- tpch.orders project ({o_orderdate:any,o_orderpriority:any}) filter on: and(ge($$o.getField(4), "1996-05-01"), lt($$o.getField(4), "1996-08-01")) range-filter on: and(ge($$o.getField(4), "1996-05-01"), lt($$o.getField(4), "1996-08-01")) [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 [$$79]  |PARTITIONED|
+                                          assign [$$86] <- [true] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- ASSIGN  |PARTITIONED|
+                                            select (lt($$l.getField(11), $$l.getField(12))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              project ([$$79, $$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                               -- STREAM_PROJECT  |PARTITIONED|
-                                                select (and(ge($$76, "1996-05-01"), lt($$76, "1996-08-01"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- STREAM_SELECT  |PARTITIONED|
-                                                  project ([$$77, $$75, $$76]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    assign [$$75, $$76] <- [$$o.getField(5), $$o.getField(4)] [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]
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        data-scan []<-[$$77, $$o] <- tpch.orders project ({o_orderdate:any,o_orderpriority:any}) filter on: and(ge($$o.getField(4), "1996-05-01"), lt($$o.getField(4), "1996-08-01")) range-filter on: and(ge($$o.getField(4), "1996-05-01"), lt($$o.getField(4), "1996-08-01")) [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 [$$79]  |PARTITIONED|
-                                              assign [$$86] <- [true] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- ASSIGN  |PARTITIONED|
-                                                select (lt($$l.getField(11), $$l.getField(12))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- STREAM_SELECT  |PARTITIONED|
-                                                  project ([$$79, $$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 []<-[$$78, $$79, $$l] <- tpch.lineitem filter on: lt($$l.getField(11), $$l.getField(12)) [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|
-                                                      data-scan []<-[$$78, $$79, $$l] <- tpch.lineitem filter on: lt($$l.getField(11), $$l.getField(12)) [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|
+                                                      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/tpch/query-plans/query-plans.07.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.07.plan
index 63771be..0b3a58b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.07.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.07.plan
@@ -2,159 +2,143 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$149]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$149] <- [{"$1": 5, "n_name": $$n_name, "revenue": $$165}] [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 [$$165(DESC) ]  |PARTITIONED|
-          order (DESC, $$165) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$165(DESC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              group by ([$$n_name := $$176]) decor ([]) {
-                        aggregate [$$165] <- [agg-global-sql-sum($$175)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$176]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- HASH_PARTITION_EXCHANGE [$$176]  |PARTITIONED|
-                  group by ([$$176 := $$150]) decor ([]) {
-                            aggregate [$$175] <- [agg-local-sql-sum(numeric-multiply($$173, numeric-subtract(1, $$174)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- AGGREGATE  |LOCAL|
-                              nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                         } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- SORT_GROUP_BY[$$150]  |PARTITIONED|
-                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      project ([$$173, $$174, $$150]) [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|
-                          join (eq($$163, $$161)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- HYBRID_HASH_JOIN [$$163][$$161]  |PARTITIONED|
-                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- HASH_PARTITION_EXCHANGE [$$163]  |PARTITIONED|
-                              project ([$$173, $$174, $$150, $$163]) [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|
-                                  join (eq($$154, $$160)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- HYBRID_HASH_JOIN [$$154][$$160]  |PARTITIONED|
-                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- HASH_PARTITION_EXCHANGE [$$154]  |PARTITIONED|
-                                      project ([$$173, $$174, $$154]) [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|
-                                          join (and(eq($$153, $$154), eq($$170, $$159))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- HYBRID_HASH_JOIN [$$153, $$170][$$154, $$159]  |PARTITIONED|
-                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- HASH_PARTITION_EXCHANGE [$$153, $$170]  |PARTITIONED|
-                                              project ([$$173, $$174, $$153, $$170]) [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|
-                                                  join (eq($$158, $$156)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- HYBRID_HASH_JOIN [$$156][$$158]  |PARTITIONED|
-                                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- HASH_PARTITION_EXCHANGE [$$156]  |PARTITIONED|
-                                                      project ([$$153, $$156]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$149] <- [{"$1": 5, "n_name": $$n_name, "revenue": $$165}] project: [$$149] [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 [$$165(DESC) ]  |PARTITIONED|
+        order (DESC, $$165) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$165(DESC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            group by ([$$n_name := $$176]) decor ([]) {
+                      aggregate [$$165] <- [agg-global-sql-sum($$175)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$176]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- HASH_PARTITION_EXCHANGE [$$176]  |PARTITIONED|
+                group by ([$$176 := $$150]) decor ([]) {
+                          aggregate [$$175] <- [agg-local-sql-sum(numeric-multiply($$173, numeric-subtract(1, $$174)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- AGGREGATE  |LOCAL|
+                            nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                       } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- SORT_GROUP_BY[$$150]  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    project ([$$173, $$174, $$150]) [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|
+                        join (eq($$163, $$161)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- HYBRID_HASH_JOIN [$$163][$$161]  |PARTITIONED|
+                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- HASH_PARTITION_EXCHANGE [$$163]  |PARTITIONED|
+                            project ([$$173, $$174, $$150, $$163]) [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|
+                                join (eq($$154, $$160)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- HYBRID_HASH_JOIN [$$154][$$160]  |PARTITIONED|
+                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- HASH_PARTITION_EXCHANGE [$$154]  |PARTITIONED|
+                                    project ([$$173, $$174, $$154]) [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|
+                                        join (and(eq($$153, $$154), eq($$170, $$159))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- HYBRID_HASH_JOIN [$$153, $$170][$$154, $$159]  |PARTITIONED|
+                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- HASH_PARTITION_EXCHANGE [$$153, $$170]  |PARTITIONED|
+                                            project ([$$173, $$174, $$153, $$170]) [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|
+                                                join (eq($$158, $$156)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                -- HYBRID_HASH_JOIN [$$156][$$158]  |PARTITIONED|
+                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                  -- HASH_PARTITION_EXCHANGE [$$156]  |PARTITIONED|
+                                                    project ([$$153, $$156]) [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|
+                                                        join (eq($$155, $$167)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                        -- HYBRID_HASH_JOIN [$$155][$$167]  |PARTITIONED|
+                                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                          -- HASH_PARTITION_EXCHANGE [$$155]  |PARTITIONED|
+                                                            assign [$$153] <- [$$c.getField(3)] project: [$$153, $$155] [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]
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                data-scan []<-[$$155, $$c] <- tpch.customer project ({c_nationkey:any}) [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 [$$167]  |PARTITIONED|
+                                                            select (and(ge($$151, "1993-01-01"), lt($$151, "1994-01-01"))) project: [$$156, $$167] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                              assign [$$167, $$151] <- [$$o.getField(1), $$o.getField(4)] project: [$$156, $$167, $$151] [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]
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  data-scan []<-[$$156, $$o] <- tpch.orders project ({o_orderdate:any,o_custkey:any}) filter on: and(ge($$o.getField(4), "1993-01-01"), lt($$o.getField(4), "1994-01-01")) range-filter on: and(ge($$o.getField(4), "1993-01-01"), lt($$o.getField(4), "1994-01-01")) [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 [$$158]  |PARTITIONED|
+                                                    assign [$$174, $$173, $$170] <- [$$l.getField(6), $$l.getField(5), $$l.getField(2)] project: [$$173, $$174, $$170, $$158] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      project ([$$158, $$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|
-                                                          join (eq($$155, $$167)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                          -- HYBRID_HASH_JOIN [$$155][$$167]  |PARTITIONED|
-                                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                            -- HASH_PARTITION_EXCHANGE [$$155]  |PARTITIONED|
-                                                              project ([$$153, $$155]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                assign [$$153] <- [$$c.getField(3)] [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]
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    data-scan []<-[$$155, $$c] <- tpch.customer project ({c_nationkey:any}) [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 [$$167]  |PARTITIONED|
-                                                              project ([$$156, $$167]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                select (and(ge($$151, "1993-01-01"), lt($$151, "1994-01-01"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                -- STREAM_SELECT  |PARTITIONED|
-                                                                  project ([$$156, $$167, $$151]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    assign [$$167, $$151] <- [$$o.getField(1), $$o.getField(4)] [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]
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        data-scan []<-[$$156, $$o] <- tpch.orders project ({o_orderdate:any,o_custkey:any}) filter on: and(ge($$o.getField(4), "1993-01-01"), lt($$o.getField(4), "1994-01-01")) range-filter on: and(ge($$o.getField(4), "1993-01-01"), lt($$o.getField(4), "1994-01-01")) [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 [$$158]  |PARTITIONED|
-                                                      project ([$$173, $$174, $$170, $$158]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        assign [$$174, $$173, $$170] <- [$$l.getField(6), $$l.getField(5), $$l.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          project ([$$158, $$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                          data-scan []<-[$$157, $$158, $$l] <- tpch.lineitem project ({l_extendedprice:any,l_suppkey:any,l_discount:any}) [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|
-                                                              data-scan []<-[$$157, $$158, $$l] <- tpch.lineitem project ({l_extendedprice:any,l_suppkey:any,l_discount:any}) [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 [$$154, $$159]  |PARTITIONED|
-                                              project ([$$154, $$159]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                assign [$$154] <- [$$s.getField(3)] [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]
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    data-scan []<-[$$159, $$s] <- tpch.supplier project ({s_nationkey:any}) [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 [$$160]  |PARTITIONED|
-                                      project ([$$150, $$163, $$160]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        assign [$$150, $$163] <- [$$n.getField(1), $$n.getField(2)] [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|
                                           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            data-scan []<-[$$160, $$n] <- tpch.nation project ({n_regionkey:any,n_name:any}) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$154, $$159]  |PARTITIONED|
+                                            assign [$$154] <- [$$s.getField(3)] project: [$$154, $$159] [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]
                                               -- 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 [$$161]  |PARTITIONED|
-                              project ([$$161]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                select (eq($$r.getField(1), "AFRICA")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |PARTITIONED|
+                                                data-scan []<-[$$159, $$s] <- tpch.supplier project ({s_nationkey:any}) [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  |PARTITIONED|
-                                    data-scan []<-[$$161, $$r] <- tpch.region project ({r_name:any}) filter on: eq($$r.getField(1), "AFRICA") range-filter on: eq($$r.getField(1), "AFRICA") [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$160]  |PARTITIONED|
+                                    assign [$$150, $$163] <- [$$n.getField(1), $$n.getField(2)] project: [$$150, $$163, $$160] [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]
                                       -- 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 []<-[$$160, $$n] <- tpch.nation project ({n_regionkey:any,n_name:any}) [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 [$$161]  |PARTITIONED|
+                            select (eq($$r.getField(1), "AFRICA")) project: [$$161] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- STREAM_SELECT  |PARTITIONED|
+                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                data-scan []<-[$$161, $$r] <- tpch.region project ({r_name:any}) filter on: eq($$r.getField(1), "AFRICA") range-filter on: eq($$r.getField(1), "AFRICA") [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_column/tpch/query-plans/query-plans.08.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.08.plan
index 638554f..b4e377b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.08.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.08.plan
@@ -2,173 +2,155 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$186]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$186] <- [{"$1": 7, "supp_nation": $$supp_nation, "cust_nation": $$cust_nation, "l_year": $$l_year, "revenue": $$200}] [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 [$$supp_nation(ASC), $$cust_nation(ASC), $$l_year(ASC) ]  |PARTITIONED|
-          group by ([$$supp_nation := $$217; $$cust_nation := $$218; $$l_year := $$219]) decor ([]) {
-                    aggregate [$$200] <- [agg-global-sql-sum($$216)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$217, $$218, $$219]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$217, $$218, $$219]  |PARTITIONED|
-              group by ([$$217 := $$191; $$218 := $$192; $$219 := $$214]) decor ([]) {
-                        aggregate [$$216] <- [agg-local-sql-sum($$184)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$191, $$192, $$214]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$184, $$191, $$192, $$214]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$184, $$214] <- [numeric-multiply($$211, numeric-subtract(1, $$212)), get-year(date($$190))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
-                      select (or(and(eq($$191, "KENYA"), eq($$192, "PERU")), and(eq($$191, "PERU"), eq($$192, "KENYA")))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STREAM_SELECT  |PARTITIONED|
-                        project ([$$191, $$211, $$212, $$190, $$192]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- STREAM_PROJECT  |PARTITIONED|
+    assign [$$186] <- [{"$1": 7, "supp_nation": $$supp_nation, "cust_nation": $$cust_nation, "l_year": $$l_year, "revenue": $$200}] project: [$$186] [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 [$$supp_nation(ASC), $$cust_nation(ASC), $$l_year(ASC) ]  |PARTITIONED|
+        group by ([$$supp_nation := $$217; $$cust_nation := $$218; $$l_year := $$219]) decor ([]) {
+                  aggregate [$$200] <- [agg-global-sql-sum($$216)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$217, $$218, $$219]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$217, $$218, $$219]  |PARTITIONED|
+            group by ([$$217 := $$191; $$218 := $$192; $$219 := $$214]) decor ([]) {
+                      aggregate [$$216] <- [agg-local-sql-sum($$184)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$191, $$192, $$214]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                assign [$$184, $$214] <- [numeric-multiply($$211, numeric-subtract(1, $$212)), get-year(date($$190))] project: [$$184, $$191, $$192, $$214] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  select (or(and(eq($$191, "KENYA"), eq($$192, "PERU")), and(eq($$191, "PERU"), eq($$192, "KENYA")))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STREAM_SELECT  |PARTITIONED|
+                    project ([$$191, $$211, $$212, $$190, $$192]) [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|
+                        join (eq($$201, $$199)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- HYBRID_HASH_JOIN [$$201][$$199]  |PARTITIONED|
                           exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            join (eq($$201, $$199)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                            -- HYBRID_HASH_JOIN [$$201][$$199]  |PARTITIONED|
-                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- HASH_PARTITION_EXCHANGE [$$201]  |PARTITIONED|
-                                project ([$$191, $$211, $$212, $$190, $$201]) [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|
-                                    join (eq($$205, $$198)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- HYBRID_HASH_JOIN [$$205][$$198]  |PARTITIONED|
-                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
-                                        project ([$$211, $$212, $$190, $$201, $$205]) [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|
-                                            join (eq($$197, $$210)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- HYBRID_HASH_JOIN [$$210][$$197]  |PARTITIONED|
-                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- HASH_PARTITION_EXCHANGE [$$210]  |PARTITIONED|
-                                                project ([$$211, $$212, $$190, $$205, $$210]) [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|
-                                                    join (eq($$196, $$195)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- HYBRID_HASH_JOIN [$$195][$$196]  |PARTITIONED|
-                                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- HASH_PARTITION_EXCHANGE [$$195]  |PARTITIONED|
-                                                        project ([$$211, $$212, $$190, $$205, $$195]) [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|
-                                                            join (eq($$193, $$204)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                            -- HYBRID_HASH_JOIN [$$193][$$204]  |PARTITIONED|
-                                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                              -- HASH_PARTITION_EXCHANGE [$$193]  |PARTITIONED|
-                                                                project ([$$205, $$193]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  assign [$$205] <- [$$s.getField(3)] [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]
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      data-scan []<-[$$193, $$s] <- tpch.supplier project ({s_nationkey:any}) [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 [$$204]  |PARTITIONED|
-                                                                select (and(ge($$190, "1995-01-01"), le($$190, "1996-12-31"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                -- STREAM_SELECT  |PARTITIONED|
-                                                                  project ([$$195, $$212, $$211, $$204, $$190]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    assign [$$212, $$211, $$204, $$190] <- [$$l.getField(6), $$l.getField(5), $$l.getField(2), $$l.getField(10)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                    -- ASSIGN  |PARTITIONED|
-                                                                      project ([$$195, $$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 []<-[$$194, $$195, $$l] <- tpch.lineitem project ({l_extendedprice:any,l_suppkey:any,l_shipdate:any,l_discount:any}) filter on: and(ge($$l.getField(10), "1995-01-01"), le($$l.getField(10), "1996-12-31")) range-filter on: and(ge($$l.getField(10), "1995-01-01"), le($$l.getField(10), "1996-12-31")) [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 [$$196]  |PARTITIONED|
-                                                        project ([$$210, $$196]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          assign [$$210] <- [$$o.getField(1)] [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]
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              data-scan []<-[$$196, $$o] <- tpch.orders project ({o_custkey:any}) [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 [$$197]  |PARTITIONED|
-                                                project ([$$201, $$197]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  assign [$$201] <- [$$c.getField(3)] [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]
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      data-scan []<-[$$197, $$c] <- tpch.customer project ({c_nationkey:any}) [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  |PARTITIONED|
-                                        replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- REPLICATE  |PARTITIONED|
-                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- HASH_PARTITION_EXCHANGE [$$198]  |PARTITIONED|
-                                            project ([$$191, $$198]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              assign [$$191] <- [$$n1.getField(1)] [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]
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  data-scan []<-[$$198, $$n1] <- tpch.nation project ({n_name:any}) [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|
+                          -- HASH_PARTITION_EXCHANGE [$$201]  |PARTITIONED|
+                            project ([$$191, $$211, $$212, $$190, $$201]) [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|
-                                project ([$$192, $$199]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  assign [$$192, $$199] <- [$$191, $$198] [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]
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- REPLICATE  |PARTITIONED|
-                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- HASH_PARTITION_EXCHANGE [$$198]  |PARTITIONED|
-                                          project ([$$191, $$198]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            assign [$$191] <- [$$n1.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                join (eq($$205, $$198)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- HYBRID_HASH_JOIN [$$205][$$198]  |PARTITIONED|
+                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
+                                    project ([$$211, $$212, $$190, $$201, $$205]) [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|
+                                        join (eq($$197, $$210)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- HYBRID_HASH_JOIN [$$210][$$197]  |PARTITIONED|
+                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- HASH_PARTITION_EXCHANGE [$$210]  |PARTITIONED|
+                                            project ([$$211, $$212, $$190, $$205, $$210]) [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|
+                                                join (eq($$196, $$195)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                -- HYBRID_HASH_JOIN [$$195][$$196]  |PARTITIONED|
+                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                  -- HASH_PARTITION_EXCHANGE [$$195]  |PARTITIONED|
+                                                    project ([$$211, $$212, $$190, $$205, $$195]) [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|
+                                                        join (eq($$193, $$204)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                        -- HYBRID_HASH_JOIN [$$193][$$204]  |PARTITIONED|
+                                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                          -- HASH_PARTITION_EXCHANGE [$$193]  |PARTITIONED|
+                                                            assign [$$205] <- [$$s.getField(3)] project: [$$205, $$193] [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]
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                data-scan []<-[$$193, $$s] <- tpch.supplier project ({s_nationkey:any}) [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 [$$204]  |PARTITIONED|
+                                                            select (and(ge($$190, "1995-01-01"), le($$190, "1996-12-31"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                              assign [$$212, $$211, $$204, $$190] <- [$$l.getField(6), $$l.getField(5), $$l.getField(2), $$l.getField(10)] project: [$$195, $$212, $$211, $$204, $$190] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                project ([$$195, $$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 []<-[$$194, $$195, $$l] <- tpch.lineitem project ({l_extendedprice:any,l_suppkey:any,l_shipdate:any,l_discount:any}) filter on: and(ge($$l.getField(10), "1995-01-01"), le($$l.getField(10), "1996-12-31")) range-filter on: and(ge($$l.getField(10), "1995-01-01"), le($$l.getField(10), "1996-12-31")) [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 [$$196]  |PARTITIONED|
+                                                    assign [$$210] <- [$$o.getField(1)] project: [$$210, $$196] [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]
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        data-scan []<-[$$196, $$o] <- tpch.orders project ({o_custkey:any}) [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 [$$197]  |PARTITIONED|
+                                            assign [$$201] <- [$$c.getField(3)] project: [$$201, $$197] [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]
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                data-scan []<-[$$198, $$n1] <- tpch.nation project ({n_name:any}) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                data-scan []<-[$$197, $$c] <- tpch.customer project ({c_nationkey:any}) [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  |PARTITIONED|
+                                    replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- REPLICATE  |PARTITIONED|
+                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- HASH_PARTITION_EXCHANGE [$$198]  |PARTITIONED|
+                                        assign [$$191] <- [$$n1.getField(1)] project: [$$191, $$198] [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]
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            data-scan []<-[$$198, $$n1] <- tpch.nation project ({n_name:any}) [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  |PARTITIONED|
+                            assign [$$192, $$199] <- [$$191, $$198] project: [$$192, $$199] [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]
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- REPLICATE  |PARTITIONED|
+                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- HASH_PARTITION_EXCHANGE [$$198]  |PARTITIONED|
+                                    assign [$$191] <- [$$n1.getField(1)] project: [$$191, $$198] [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]
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        data-scan []<-[$$198, $$n1] <- tpch.nation project ({n_name:any}) [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_column/tpch/query-plans/query-plans.09.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.09.plan
index f96fc1f..0b918a8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.09.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.09.plan
@@ -2,209 +2,187 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$199]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$199] <- [{"$1": 8, "o_year": $$o_year, "mkt_share": numeric-divide($$214, $$215)}] [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]
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          group by ([$$o_year := $$238]) decor ([]) {
-                    aggregate [$$214, $$215] <- [agg-global-sql-sum($$236), agg-global-sql-sum($$237)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- AGGREGATE  |LOCAL|
-                      nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                 } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- SORT_GROUP_BY[$$238]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- HASH_PARTITION_EXCHANGE [$$238]  |PARTITIONED|
-              group by ([$$238 := $$200]) decor ([]) {
-                        aggregate [$$236, $$237] <- [agg-local-sql-sum(switch-case(true, eq($$232, "PERU"), numeric-multiply($$230, numeric-subtract(1, $$231)), 0)), agg-local-sql-sum(numeric-multiply($$230, numeric-subtract(1, $$231)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- SORT_GROUP_BY[$$200]  |PARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$232, $$230, $$231, $$200]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    assign [$$199] <- [{"$1": 8, "o_year": $$o_year, "mkt_share": numeric-divide($$214, $$215)}] project: [$$199] [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]
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        group by ([$$o_year := $$238]) decor ([]) {
+                  aggregate [$$214, $$215] <- [agg-global-sql-sum($$236), agg-global-sql-sum($$237)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- SORT_GROUP_BY[$$238]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- HASH_PARTITION_EXCHANGE [$$238]  |PARTITIONED|
+            group by ([$$238 := $$200]) decor ([]) {
+                      aggregate [$$236, $$237] <- [agg-local-sql-sum(switch-case(true, eq($$232, "PERU"), numeric-multiply($$230, numeric-subtract(1, $$231)), 0)), agg-local-sql-sum(numeric-multiply($$230, numeric-subtract(1, $$231)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- AGGREGATE  |LOCAL|
+                        nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                   } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- SORT_GROUP_BY[$$200]  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                assign [$$200] <- [get-year(date($$201))] project: [$$232, $$230, $$231, $$200] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$232, $$230, $$231, $$201]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$200] <- [get-year(date($$201))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ASSIGN  |PARTITIONED|
-                      project ([$$232, $$230, $$231, $$201]) [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|
+                      join (eq($$212, $$210)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- HYBRID_HASH_JOIN [$$212][$$210]  |PARTITIONED|
                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          join (eq($$212, $$210)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- HYBRID_HASH_JOIN [$$212][$$210]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$212]  |PARTITIONED|
+                          project ([$$232, $$230, $$231, $$201, $$212]) [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]
-                            -- HASH_PARTITION_EXCHANGE [$$212]  |PARTITIONED|
-                              project ([$$232, $$230, $$231, $$201, $$212]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              join (eq($$216, $$209)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- HYBRID_HASH_JOIN [$$216][$$209]  |PARTITIONED|
                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  join (eq($$216, $$209)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- HYBRID_HASH_JOIN [$$216][$$209]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$216]  |PARTITIONED|
+                                  project ([$$230, $$231, $$201, $$212, $$216]) [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]
-                                    -- HASH_PARTITION_EXCHANGE [$$216]  |PARTITIONED|
-                                      project ([$$230, $$231, $$201, $$212, $$216]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      join (eq($$218, $$208)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- HYBRID_HASH_JOIN [$$218][$$208]  |PARTITIONED|
                                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          join (eq($$218, $$208)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- HYBRID_HASH_JOIN [$$218][$$208]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$218]  |PARTITIONED|
+                                          project ([$$230, $$231, $$201, $$216, $$218]) [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]
-                                            -- HASH_PARTITION_EXCHANGE [$$218]  |PARTITIONED|
-                                              project ([$$230, $$231, $$201, $$216, $$218]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              join (eq($$226, $$207)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- HYBRID_HASH_JOIN [$$226][$$207]  |PARTITIONED|
                                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  join (eq($$226, $$207)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- HYBRID_HASH_JOIN [$$226][$$207]  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$226]  |PARTITIONED|
+                                                  project ([$$230, $$231, $$201, $$216, $$226]) [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]
-                                                    -- HASH_PARTITION_EXCHANGE [$$226]  |PARTITIONED|
-                                                      project ([$$230, $$231, $$201, $$216, $$226]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      join (eq($$205, $$206)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                      -- HYBRID_HASH_JOIN [$$205][$$206]  |PARTITIONED|
                                                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          join (eq($$205, $$206)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                          -- HYBRID_HASH_JOIN [$$205][$$206]  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
+                                                          project ([$$230, $$231, $$216, $$205]) [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]
-                                                            -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
-                                                              project ([$$230, $$231, $$216, $$205]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              join (and(eq($$202, $$221), eq($$203, $$223))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                              -- HYBRID_HASH_JOIN [$$202, $$203][$$221, $$223]  |PARTITIONED|
                                                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  join (and(eq($$202, $$221), eq($$203, $$223))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                  -- HYBRID_HASH_JOIN [$$202, $$203][$$221, $$223]  |PARTITIONED|
-                                                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                    -- HASH_PARTITION_EXCHANGE [$$202, $$203]  |PARTITIONED|
-                                                                      join (true) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                      -- NESTED_LOOP  |PARTITIONED|
-                                                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          project ([$$202]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            select (eq($$p.getField(4), "ECONOMY BURNISHED NICKEL")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                            -- STREAM_SELECT  |PARTITIONED|
-                                                                              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                data-scan []<-[$$202, $$p] <- tpch.part project ({p_type:any}) filter on: eq($$p.getField(4), "ECONOMY BURNISHED NICKEL") range-filter on: eq($$p.getField(4), "ECONOMY BURNISHED NICKEL") [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]
-                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                          project ([$$216, $$203]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            assign [$$216] <- [$$s.getField(3)] [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]
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                data-scan []<-[$$203, $$s] <- tpch.supplier project ({s_nationkey:any}) [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 [$$221, $$223]  |PARTITIONED|
-                                                                      project ([$$230, $$231, $$205, $$221, $$223]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                        assign [$$231, $$230, $$223, $$221] <- [$$l.getField(6), $$l.getField(5), $$l.getField(2), $$l.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                        -- ASSIGN  |PARTITIONED|
-                                                                          project ([$$205, $$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 []<-[$$204, $$205, $$l] <- tpch.lineitem project ({l_partkey:any,l_extendedprice:any,l_suppkey:any,l_discount:any}) [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 [$$206]  |PARTITIONED|
-                                                              select (and(ge($$201, "1995-01-01"), le($$201, "1996-12-31"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                              -- STREAM_SELECT  |PARTITIONED|
-                                                                project ([$$206, $$226, $$201]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  assign [$$226, $$201] <- [$$o.getField(1), $$o.getField(4)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                  -- ASSIGN  |PARTITIONED|
+                                                                -- HASH_PARTITION_EXCHANGE [$$202, $$203]  |PARTITIONED|
+                                                                  join (true) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                  -- NESTED_LOOP  |PARTITIONED|
                                                                     exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      data-scan []<-[$$206, $$o] <- tpch.orders project ({o_orderdate:any,o_custkey:any}) filter on: and(ge($$o.getField(4), "1995-01-01"), le($$o.getField(4), "1996-12-31")) range-filter on: and(ge($$o.getField(4), "1995-01-01"), le($$o.getField(4), "1996-12-31")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                      select (eq($$p.getField(4), "ECONOMY BURNISHED NICKEL")) project: [$$202] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                      -- STREAM_SELECT  |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 [$$207]  |PARTITIONED|
-                                                      project ([$$218, $$207]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        assign [$$218] <- [$$c.getField(3)] [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]
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            data-scan []<-[$$207, $$c] <- tpch.customer project ({c_nationkey:any}) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                          data-scan []<-[$$202, $$p] <- tpch.part project ({p_type:any}) filter on: eq($$p.getField(4), "ECONOMY BURNISHED NICKEL") range-filter on: eq($$p.getField(4), "ECONOMY BURNISHED NICKEL") [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]
+                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                      assign [$$216] <- [$$s.getField(3)] project: [$$216, $$203] [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]
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          data-scan []<-[$$203, $$s] <- tpch.supplier project ({s_nationkey:any}) [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 [$$221, $$223]  |PARTITIONED|
+                                                                  assign [$$231, $$230, $$223, $$221] <- [$$l.getField(6), $$l.getField(5), $$l.getField(2), $$l.getField(1)] project: [$$230, $$231, $$205, $$221, $$223] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    project ([$$205, $$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 []<-[$$204, $$205, $$l] <- tpch.lineitem project ({l_partkey:any,l_extendedprice:any,l_suppkey:any,l_discount:any}) [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 [$$206]  |PARTITIONED|
+                                                          select (and(ge($$201, "1995-01-01"), le($$201, "1996-12-31"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                            assign [$$226, $$201] <- [$$o.getField(1), $$o.getField(4)] project: [$$206, $$226, $$201] [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]
                                                               -- 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 []<-[$$206, $$o] <- tpch.orders project ({o_orderdate:any,o_custkey:any}) filter on: and(ge($$o.getField(4), "1995-01-01"), le($$o.getField(4), "1996-12-31")) range-filter on: and(ge($$o.getField(4), "1995-01-01"), le($$o.getField(4), "1996-12-31")) [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 [$$207]  |PARTITIONED|
+                                                  assign [$$218] <- [$$c.getField(3)] project: [$$218, $$207] [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]
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      data-scan []<-[$$207, $$c] <- tpch.customer project ({c_nationkey:any}) [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 [$$208]  |PARTITIONED|
+                                          assign [$$212] <- [$$n1.getField(2)] project: [$$212, $$208] [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]
-                                            -- HASH_PARTITION_EXCHANGE [$$208]  |PARTITIONED|
-                                              project ([$$212, $$208]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                assign [$$212] <- [$$n1.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]
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- REPLICATE  |PARTITIONED|
-                                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        data-scan []<-[$$208, $$n1] <- tpch.nation project ({n_regionkey:any,n_name:any}) [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 [$$209]  |PARTITIONED|
-                                      project ([$$232, $$209]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        assign [$$232] <- [$$n2.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- ASSIGN  |PARTITIONED|
-                                          project ([$$209, $$n2]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            assign [$$209, $$n2] <- [$$208, $$n1] [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]
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- REPLICATE  |PARTITIONED|
-                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    data-scan []<-[$$208, $$n1] <- tpch.nation project ({n_regionkey:any,n_name:any}) [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 [$$210]  |PARTITIONED|
-                              project ([$$210]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                select (eq($$r.getField(1), "AMERICA")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- STREAM_SELECT  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    data-scan []<-[$$210, $$r] <- tpch.region project ({r_name:any}) filter on: eq($$r.getField(1), "AMERICA") range-filter on: eq($$r.getField(1), "AMERICA") [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- REPLICATE  |PARTITIONED|
+                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  data-scan []<-[$$208, $$n1] <- tpch.nation project ({n_regionkey:any,n_name:any}) [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 [$$209]  |PARTITIONED|
+                                  assign [$$232] <- [$$n2.getField(1)] project: [$$232, $$209] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- ASSIGN  |PARTITIONED|
+                                    assign [$$209, $$n2] <- [$$208, $$n1] project: [$$209, $$n2] [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]
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- REPLICATE  |PARTITIONED|
+                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            data-scan []<-[$$208, $$n1] <- tpch.nation project ({n_regionkey:any,n_name:any}) [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 [$$210]  |PARTITIONED|
+                          select (eq($$r.getField(1), "AMERICA")) project: [$$210] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- STREAM_SELECT  |PARTITIONED|
+                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              data-scan []<-[$$210, $$r] <- tpch.region project ({r_name:any}) filter on: eq($$r.getField(1), "AMERICA") range-filter on: eq($$r.getField(1), "AMERICA") [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_column/tpch/query-plans/query-plans.10.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.10.plan
index 4483188..e0fc455 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.10.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.10.plan
@@ -2,125 +2,117 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$122]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$122] <- [{"$1": 9, "$2": $$140}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$140] <- [agg-sql-sum($$144)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- AGGREGATE  |UNPARTITIONED|
-          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$144] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- AGGREGATE  |PARTITIONED|
-              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                join (eq($$138, $$137)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- HYBRID_HASH_JOIN [$$138][$$137]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- HASH_PARTITION_EXCHANGE [$$138]  |PARTITIONED|
-                    project ([$$138]) [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|
-                        join (eq($$136, $$132)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HYBRID_HASH_JOIN [$$132][$$136]  |PARTITIONED|
-                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- HASH_PARTITION_EXCHANGE [$$132]  |PARTITIONED|
-                            project ([$$138, $$132]) [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|
-                                join (and(eq($$130, $$135), eq($$134, $$125))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- HYBRID_HASH_JOIN [$$130, $$125][$$135, $$134]  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    project ([$$130, $$138, $$132, $$125]) [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|
-                                        join (and(eq($$130, $$128), eq($$129, $$125))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- HYBRID_HASH_JOIN [$$130, $$129][$$128, $$125]  |PARTITIONED|
-                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- HASH_PARTITION_EXCHANGE [$$130, $$129]  |PARTITIONED|
-                                            join (true) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- NESTED_LOOP  |PARTITIONED|
+    assign [$$122] <- [{"$1": 9, "$2": $$140}] project: [$$122] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      aggregate [$$140] <- [agg-sql-sum($$144)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- AGGREGATE  |UNPARTITIONED|
+        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+          aggregate [$$144] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- AGGREGATE  |PARTITIONED|
+            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              join (eq($$138, $$137)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- HYBRID_HASH_JOIN [$$138][$$137]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- HASH_PARTITION_EXCHANGE [$$138]  |PARTITIONED|
+                  project ([$$138]) [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|
+                      join (eq($$136, $$132)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- HYBRID_HASH_JOIN [$$132][$$136]  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- HASH_PARTITION_EXCHANGE [$$132]  |PARTITIONED|
+                          project ([$$138, $$132]) [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|
+                              join (and(eq($$130, $$135), eq($$134, $$125))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- HYBRID_HASH_JOIN [$$130, $$125][$$135, $$134]  |PARTITIONED|
+                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  project ([$$130, $$138, $$132, $$125]) [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|
+                                      join (and(eq($$130, $$128), eq($$129, $$125))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- HYBRID_HASH_JOIN [$$130, $$129][$$128, $$125]  |PARTITIONED|
+                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- HASH_PARTITION_EXCHANGE [$$130, $$129]  |PARTITIONED|
+                                          join (true) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- NESTED_LOOP  |PARTITIONED|
+                                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              select (like($$p.getField(1), "%green%")) project: [$$129] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  data-scan []<-[$$129, $$p] <- tpch.part project ({p_name:any}) filter on: like($$p.getField(1), "%green%") [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]
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              assign [$$138] <- [$$s.getField(3)] project: [$$130, $$138] [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]
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  data-scan []<-[$$130, $$s] <- tpch.supplier project ({s_nationkey:any}) [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 [$$128, $$125]  |PARTITIONED|
+                                          assign [$$128, $$125] <- [$$l.getField(2), $$l.getField(1)] project: [$$132, $$128, $$125] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- ASSIGN  |PARTITIONED|
+                                            project ([$$132, $$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|
-                                                project ([$$129]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  select (like($$p.getField(1), "%green%")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- STREAM_SELECT  |PARTITIONED|
-                                                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      data-scan []<-[$$129, $$p] <- tpch.part project ({p_name:any}) filter on: like($$p.getField(1), "%green%") [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]
-                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                project ([$$130, $$138]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  assign [$$138] <- [$$s.getField(3)] [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]
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      data-scan []<-[$$130, $$s] <- tpch.supplier project ({s_nationkey:any}) [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 [$$128, $$125]  |PARTITIONED|
-                                            project ([$$132, $$128, $$125]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              assign [$$128, $$125] <- [$$l.getField(2), $$l.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- ASSIGN  |PARTITIONED|
-                                                project ([$$132, $$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                data-scan []<-[$$131, $$132, $$l] <- tpch.lineitem project ({l_partkey:any,l_suppkey:any}) [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|
-                                                    data-scan []<-[$$131, $$132, $$l] <- tpch.lineitem project ({l_partkey:any,l_suppkey:any}) [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 [$$135, $$134]  |PARTITIONED|
-                                    project ([$$134, $$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 []<-[$$133, $$134, $$135, $$ps] <- tpch.partsupp project ({}) [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 [$$136]  |PARTITIONED|
-                            project ([$$136]) [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 []<-[$$136, $$o] <- tpch.orders project ({}) [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 [$$137]  |PARTITIONED|
-                    project ([$$137]) [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 []<-[$$137, $$n] <- tpch.nation project ({}) [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|
+                                                    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 [$$135, $$134]  |PARTITIONED|
+                                  project ([$$134, $$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 []<-[$$133, $$134, $$135, $$ps] <- tpch.partsupp project ({}) [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 [$$136]  |PARTITIONED|
+                          project ([$$136]) [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 []<-[$$136, $$o] <- tpch.orders project ({}) [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 [$$137]  |PARTITIONED|
+                  project ([$$137]) [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 []<-[$$137, $$n] <- tpch.nation project ({}) [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_column/tpch/query-plans/query-plans.11.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.11.plan
index 2504041..cf33f7b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.11.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/tpch/query-plans/query-plans.11.plan
@@ -4,121 +4,109 @@
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit 20 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$185]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$185] <- [{"$1": 10, "c_custkey": $$c_custkey, "c_name": $$c_name, "revenue": $$203, "c_acctbal": $$c_acctbal, "n_name": $$n_name, "c_address": $$c_address, "c_phone": $$c_phone, "c_comment": $$c_comment}] [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 [$$203(DESC) ]  |PARTITIONED|
-            limit 20 [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]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (topK: 20) (DESC, $$203) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STABLE_SORT [topK: 20] [$$203(DESC)]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    group by ([$$c_custkey := $$211; $$n_name := $$215]) decor ([$$c_name := $$212; $$c_acctbal := $$213; $$c_phone := $$214; $$c_address := $$216; $$c_comment := $$217]) {
-                              aggregate [$$203] <- [agg-global-sql-sum($$210)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- AGGREGATE  |LOCAL|
-                                nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                           } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- SORT_GROUP_BY[$$211, $$215]  |PARTITIONED|
-                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- HASH_PARTITION_EXCHANGE [$$211, $$215]  |PARTITIONED|
-                        group by ([$$211 := $$195; $$215 := $$190]) decor ([$$212 := $$187; $$213 := $$188; $$214 := $$189; $$216 := $$191; $$217 := $$192]) {
-                                  aggregate [$$210] <- [agg-local-sql-sum(numeric-multiply($$208, numeric-subtract(1, $$209)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- AGGREGATE  |LOCAL|
-                                    nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                               } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- SORT_GROUP_BY[$$195, $$190]  |PARTITIONED|
-                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            project ([$$208, $$209, $$195, $$187, $$188, $$189, $$190, $$191, $$192]) [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|
-                                join (eq($$201, $$199)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- HYBRID_HASH_JOIN [$$201][$$199]  |PARTITIONED|
-                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- HASH_PARTITION_EXCHANGE [$$201]  |PARTITIONED|
-                                    project ([$$208, $$209, $$195, $$187, $$188, $$189, $$191, $$192, $$201]) [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|
-                                        join (eq($$198, $$196)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                        -- HYBRID_HASH_JOIN [$$196][$$198]  |PARTITIONED|
-                                          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                          -- HASH_PARTITION_EXCHANGE [$$196]  |PARTITIONED|
-                                            project ([$$195, $$187, $$188, $$189, $$191, $$192, $$201, $$196]) [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|
-                                                join (eq($$195, $$206)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- HYBRID_HASH_JOIN [$$195][$$206]  |PARTITIONED|
-                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- HASH_PARTITION_EXCHANGE [$$195]  |PARTITIONED|
-                                                    project ([$$195, $$187, $$188, $$189, $$191, $$192, $$201]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      assign [$$192, $$191, $$189, $$188, $$187, $$201] <- [$$c.getField(7), $$c.getField(2), $$c.getField(4), $$c.getField(5), $$c.getField(1), $$c.getField(3)] [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]
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          data-scan []<-[$$195, $$c] <- tpch.customer project ({c_phone:any,c_acctbal:any,c_address:any,c_nationkey:any,c_name:any,c_comment:any}) [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 [$$206]  |PARTITIONED|
-                                                    project ([$$196, $$206]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      select (and(lt($$193, "1993-10-01"), ge($$193, "1993-07-01"))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                        project ([$$196, $$206, $$193]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          assign [$$206, $$193] <- [$$o.getField(1), $$o.getField(4)] [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]
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              data-scan []<-[$$196, $$o] <- tpch.orders project ({o_orderdate:any,o_custkey:any}) filter on: and(lt($$o.getField(4), "1993-10-01"), ge($$o.getField(4), "1993-07-01")) range-filter on: and(lt($$o.getField(4), "1993-10-01"), ge($$o.getField(4), "1993-07-01")) [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 [$$198]  |PARTITIONED|
-                                            project ([$$208, $$209, $$198]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              select (eq($$l.getField(8), "R")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                              -- STREAM_SELECT  |PARTITIONED|
-                                                assign [$$209, $$208] <- [$$l.getField(6), $$l.getField(5)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- ASSIGN  |PARTITIONED|
-                                                  project ([$$198, $$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- STREAM_PROJECT  |PARTITIONED|
+      assign [$$185] <- [{"$1": 10, "c_custkey": $$c_custkey, "c_name": $$c_name, "revenue": $$203, "c_acctbal": $$c_acctbal, "n_name": $$n_name, "c_address": $$c_address, "c_phone": $$c_phone, "c_comment": $$c_comment}] project: [$$185] [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 [$$203(DESC) ]  |PARTITIONED|
+          limit 20 [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]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              order (topK: 20) (DESC, $$203) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STABLE_SORT [topK: 20] [$$203(DESC)]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  group by ([$$c_custkey := $$211; $$n_name := $$215]) decor ([$$c_name := $$212; $$c_acctbal := $$213; $$c_phone := $$214; $$c_address := $$216; $$c_comment := $$217]) {
+                            aggregate [$$203] <- [agg-global-sql-sum($$210)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- AGGREGATE  |LOCAL|
+                              nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                         } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- SORT_GROUP_BY[$$211, $$215]  |PARTITIONED|
+                    exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- HASH_PARTITION_EXCHANGE [$$211, $$215]  |PARTITIONED|
+                      group by ([$$211 := $$195; $$215 := $$190]) decor ([$$212 := $$187; $$213 := $$188; $$214 := $$189; $$216 := $$191; $$217 := $$192]) {
+                                aggregate [$$210] <- [agg-local-sql-sum(numeric-multiply($$208, numeric-subtract(1, $$209)))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- AGGREGATE  |LOCAL|
+                                  nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                             } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- SORT_GROUP_BY[$$195, $$190]  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          project ([$$208, $$209, $$195, $$187, $$188, $$189, $$190, $$191, $$192]) [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|
+                              join (eq($$201, $$199)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- HYBRID_HASH_JOIN [$$201][$$199]  |PARTITIONED|
+                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- HASH_PARTITION_EXCHANGE [$$201]  |PARTITIONED|
+                                  project ([$$208, $$209, $$195, $$187, $$188, $$189, $$191, $$192, $$201]) [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|
+                                      join (eq($$198, $$196)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- HYBRID_HASH_JOIN [$$196][$$198]  |PARTITIONED|
+                                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                        -- HASH_PARTITION_EXCHANGE [$$196]  |PARTITIONED|
+                                          project ([$$195, $$187, $$188, $$189, $$191, $$192, $$201, $$196]) [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|
+                                              join (eq($$195, $$206)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                              -- HYBRID_HASH_JOIN [$$195][$$206]  |PARTITIONED|
+                                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                -- HASH_PARTITION_EXCHANGE [$$195]  |PARTITIONED|
+                                                  assign [$$192, $$191, $$189, $$188, $$187, $$201] <- [$$c.getField(7), $$c.getField(2), $$c.getField(4), $$c.getField(5), $$c.getField(1), $$c.getField(3)] project: [$$195, $$187, $$188, $$189, $$191, $$192, $$201] [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]
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      data-scan []<-[$$197, $$198, $$l] <- tpch.lineitem project ({l_returnflag:any,l_extendedprice:any,l_discount:any}) filter on: eq($$l.getField(8), "R") range-filter on: eq($$l.getField(8), "R") [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                      data-scan []<-[$$195, $$c] <- tpch.customer project ({c_phone:any,c_acctbal:any,c_address:any,c_nationkey:any,c_name:any,c_comment:any}) [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 [$$199]  |PARTITIONED|
-                                    project ([$$190, $$199]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      assign [$$190] <- [$$n.getField(1)] [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]
+                                                -- HASH_PARTITION_EXCHANGE [$$206]  |PARTITIONED|
+                                                  select (and(lt($$193, "1993-10-01"), ge($$193, "1993-07-01"))) project: [$$196, $$206] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                    assign [$$206, $$193] <- [$$o.getField(1), $$o.getField(4)] project: [$$196, $$206, $$193] [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]
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        data-scan []<-[$$196, $$o] <- tpch.orders project ({o_orderdate:any,o_custkey:any}) filter on: and(lt($$o.getField(4), "1993-10-01"), ge($$o.getField(4), "1993-07-01")) range-filter on: and(lt($$o.getField(4), "1993-10-01"), ge($$o.getField(4), "1993-07-01")) [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 [$$198]  |PARTITIONED|
+                                          select (eq($$l.getField(8), "R")) project: [$$208, $$209, $$198] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            assign [$$209, $$208] <- [$$l.getField(6), $$l.getField(5)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                            -- ASSIGN  |PARTITIONED|
+                                              project ([$$198, $$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 []<-[$$197, $$198, $$l] <- tpch.lineitem project ({l_returnflag:any,l_extendedprice:any,l_discount:any}) filter on: eq($$l.getField(8), "R") range-filter on: eq($$l.getField(8), "R") [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 [$$199]  |PARTITIONED|
+                                  assign [$$190] <- [$$n.getField(1)] project: [$$190, $$199] [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]
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      data-scan []<-[$$199, $$n] <- tpch.nation project ({n_name:any}) [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|
-                                          data-scan []<-[$$199, $$n] <- tpch.nation project ({n_name:any}) [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|
+                                          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/view/view-pushdown/view-pushdown.08.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/view/view-pushdown/view-pushdown.08.plan
index a62908e..e131968 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/view/view-pushdown/view-pushdown.08.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/view/view-pushdown/view-pushdown.08.plan
@@ -2,31 +2,27 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$88]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$88] <- [{"id": $$91, "review": $$95}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-          order (ASC, $$91) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$91(ASC)]  |UNPARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-              limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_LIMIT  |UNPARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                  limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_LIMIT  |PARTITIONED|
-                    project ([$$91, $$95]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$95] <- [$$d.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
+    assign [$$88] <- [{"id": $$91, "review": $$95}] project: [$$88] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+        order (ASC, $$91) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$91(ASC)]  |UNPARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+            limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_LIMIT  |UNPARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STREAM_LIMIT  |PARTITIONED|
+                  assign [$$95] <- [$$d.getField(1)] project: [$$91, $$95] [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]
+                    -- 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 project ({year:any,review:any,quarter:any}) filter on: eq($$d.getField(1), "good") range-filter on: eq($$d.getField(1), "good") [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|
-                          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 project ({year:any,review:any,quarter:any}) filter on: eq($$d.getField(1), "good") range-filter on: eq($$d.getField(1), "good") [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|
+                          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/view/view-pushdown/view-pushdown.10.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/view/view-pushdown/view-pushdown.10.plan
index 2f69064..f3c702a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/view/view-pushdown/view-pushdown.10.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/view/view-pushdown/view-pushdown.10.plan
@@ -4,29 +4,25 @@
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$63]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$63] <- [{"id": $$65, "review": $$69}] [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 [$$65(ASC) ]  |PARTITIONED|
-            limit 3 [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]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (topK: 3) (ASC, $$65) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STABLE_SORT [topK: 3] [$$65(ASC)]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$65, $$69]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$69] <- [$$d.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
+      assign [$$63] <- [{"id": $$65, "review": $$69}] project: [$$63] [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 [$$65(ASC) ]  |PARTITIONED|
+          limit 3 [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]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              order (topK: 3) (ASC, $$65) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STABLE_SORT [topK: 3] [$$65(ASC)]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  assign [$$69] <- [$$d.getField(1)] project: [$$65, $$69] [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]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$65, $$d] <- test.DatasetWithKnownField project ({review:any}) [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|
-                          data-scan []<-[$$65, $$d] <- test.DatasetWithKnownField project ({review:any}) [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|
+                          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/view/view-pushdown/view-pushdown.12.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/view/view-pushdown/view-pushdown.12.plan
index 2486ce1..2d2888a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/view/view-pushdown/view-pushdown.12.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/view/view-pushdown/view-pushdown.12.plan
@@ -2,31 +2,27 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$94]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$94] <- [{"id": $$97, "review": $$101}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
-        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-          order (ASC, $$97) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$97(ASC)]  |UNPARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-              limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_LIMIT  |UNPARTITIONED|
-                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                  limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- STREAM_LIMIT  |PARTITIONED|
-                    project ([$$97, $$101]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$101] <- [$$d.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
+    assign [$$94] <- [{"id": $$97, "review": $$101}] project: [$$94] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+        order (ASC, $$97) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$97(ASC)]  |UNPARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+            limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_LIMIT  |UNPARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STREAM_LIMIT  |PARTITIONED|
+                  assign [$$101] <- [$$d.getField(1)] project: [$$97, $$101] [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]
+                    -- 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 project ({year:any,review:any,quarter:any}) filter on: eq($$d.getField(1), "good") range-filter on: eq($$d.getField(1), "good") [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|
-                          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 project ({year:any,review:any,quarter:any}) filter on: eq($$d.getField(1), "good") range-filter on: eq($$d.getField(1), "good") [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|
+                          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/view/view-pushdown/view-pushdown.14.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/view/view-pushdown/view-pushdown.14.plan
index 70e13e6..3b7489f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_column/view/view-pushdown/view-pushdown.14.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_column/view/view-pushdown/view-pushdown.14.plan
@@ -4,29 +4,25 @@
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit 3 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$69]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$69] <- [{"id": $$71, "review": $$75}] [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 [$$71(ASC) ]  |PARTITIONED|
-            limit 3 [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]
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (topK: 3) (ASC, $$71) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STABLE_SORT [topK: 3] [$$71(ASC)]  |PARTITIONED|
-                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$71, $$75]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$75] <- [$$d.getField(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ASSIGN  |PARTITIONED|
+      assign [$$69] <- [{"id": $$71, "review": $$75}] project: [$$69] [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 [$$71(ASC) ]  |PARTITIONED|
+          limit 3 [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]
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              order (topK: 3) (ASC, $$71) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- STABLE_SORT [topK: 3] [$$71(ASC)]  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  assign [$$75] <- [$$d.getField(1)] project: [$$71, $$75] [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]
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$71, $$d] <- test.DatasetWithKnownField project ({review:any}) [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|
-                          data-scan []<-[$$71, $$d] <- test.DatasetWithKnownField project ({review:any}) [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|
+                          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_full_parallelism/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.008.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_full_parallelism/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.008.plan
index 07a32cf..88abba5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_full_parallelism/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.008.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_full_parallelism/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.008.plan
@@ -2,81 +2,41 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$52] <- [{"t1_id": $$73, "t2_id": $$54}] [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 [$$73(ASC), $$54(ASC) ]  |PARTITIONED|
-          order (ASC, $$73) (ASC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$73(ASC), $$54(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$73, $$54]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select ($$74) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  window-aggregate [$$74] <- [win-mark-first-missing-impl($$54)] partition [$$73] order (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- WINDOW_STREAM  |PARTITIONED|
+    assign [$$52] <- [{"t1_id": $$73, "t2_id": $$54}] project: [$$52] [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 [$$73(ASC), $$54(ASC) ]  |PARTITIONED|
+        order (ASC, $$73) (ASC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$73(ASC), $$54(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select ($$74) project: [$$73, $$54] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              window-aggregate [$$74] <- [win-mark-first-missing-impl($$54)] partition [$$73] order (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- WINDOW_STREAM  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$73) (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STABLE_SORT [$$73(ASC), $$54(DESC)]  |PARTITIONED|
                     exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      order (ASC, $$73) (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STABLE_SORT [$$73(ASC), $$54(DESC)]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$73]  |PARTITIONED|
+                      union ($$70, $$61, $$54) ($$53, $$53, $$73) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- UNION_ALL  |PARTITIONED|
                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HASH_PARTITION_EXCHANGE [$$73]  |PARTITIONED|
-                          union ($$70, $$61, $$54) ($$53, $$53, $$73) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- UNION_ALL  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          project ([$$70, $$53]) [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|
-                              project ([$$70, $$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
+                            -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                              select (eq($$55, $$71.getField(1))) retain-untrue ($$70 <- missing) project: [$$53, $$61, $$70] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- STREAM_SELECT  |PARTITIONED|
                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                                  project ([$$53, $$61, $$70]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    select (eq($$55, $$71.getField(1))) retain-untrue ($$70 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- STREAM_SELECT  |PARTITIONED|
-                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        left-outer-unnest-map [$$70, $$71] <- index-search("ds_inner_typed", 0, "Default", "test", "ds_inner_typed", true, false, 1, $$61, 1, $$61, true, true, true) [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|
-                                            project ([$$53, $$55, $$61]) [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|
-                                                split ($$62) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- SPLIT  |PARTITIONED|
-                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    left-outer-unnest-map [$$60, $$61, $$62] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_typed", true, true, 1, $$55, 1, $$55, true, true, true) [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]
-                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                        project ([$$53, $$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          assign [$$55] <- [int32($$t1.getField("c_int32"))] [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]
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped [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  |PARTITIONED|
-                              project ([$$61, $$53]) [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]
-                                -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                                  project ([$$53, $$61]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    select (eq($$55, $$60)) retain-untrue ($$61 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- STREAM_SELECT  |PARTITIONED|
-                                      project ([$$53, $$55, $$60, $$61]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  left-outer-unnest-map [$$70, $$71] <- index-search("ds_inner_typed", 0, "Default", "test", "ds_inner_typed", true, false, 1, $$61, 1, $$61, true, true, true) [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|
+                                      project ([$$53, $$55, $$61]) [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|
@@ -88,15 +48,43 @@
                                               -- BTREE_SEARCH  |PARTITIONED|
                                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                                 -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                  project ([$$53, $$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    assign [$$55] <- [int32($$t1.getField("c_int32"))] [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]
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped [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|
+                                                  assign [$$55] <- [int32($$t1.getField("c_int32"))] project: [$$53, $$55] [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]
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped [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  |PARTITIONED|
+                          project ([$$61, $$53]) [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]
+                            -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                              select (eq($$55, $$60)) retain-untrue ($$61 <- missing) project: [$$53, $$61] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- STREAM_SELECT  |PARTITIONED|
+                                project ([$$53, $$55, $$60, $$61]) [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|
+                                    split ($$62) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- SPLIT  |PARTITIONED|
+                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        left-outer-unnest-map [$$60, $$61, $$62] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_typed", true, true, 1, $$55, 1, $$55, true, true, true) [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]
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            assign [$$55] <- [int32($$t1.getField("c_int32"))] project: [$$53, $$55] [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]
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped [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_full_parallelism/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.010.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_full_parallelism/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.010.plan
index 2845b04..fe416df 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_full_parallelism/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.010.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_full_parallelism/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.010.plan
@@ -2,81 +2,41 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$52] <- [{"t1_id": $$73, "t2_id": $$54}] [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 [$$73(ASC), $$54(ASC) ]  |PARTITIONED|
-          order (ASC, $$73) (ASC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$73(ASC), $$54(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$73, $$54]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select ($$74) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  window-aggregate [$$74] <- [win-mark-first-missing-impl($$54)] partition [$$73] order (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- WINDOW_STREAM  |PARTITIONED|
+    assign [$$52] <- [{"t1_id": $$73, "t2_id": $$54}] project: [$$52] [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 [$$73(ASC), $$54(ASC) ]  |PARTITIONED|
+        order (ASC, $$73) (ASC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$73(ASC), $$54(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select ($$74) project: [$$73, $$54] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              window-aggregate [$$74] <- [win-mark-first-missing-impl($$54)] partition [$$73] order (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- WINDOW_STREAM  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$73) (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STABLE_SORT [$$73(ASC), $$54(DESC)]  |PARTITIONED|
                     exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      order (ASC, $$73) (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STABLE_SORT [$$73(ASC), $$54(DESC)]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$73]  |PARTITIONED|
+                      union ($$70, $$61, $$54) ($$53, $$53, $$73) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- UNION_ALL  |PARTITIONED|
                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HASH_PARTITION_EXCHANGE [$$73]  |PARTITIONED|
-                          union ($$70, $$61, $$54) ($$53, $$53, $$73) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- UNION_ALL  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          project ([$$70, $$53]) [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|
-                              project ([$$70, $$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
+                            -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                              select (eq($$55, $$71.getField(1))) retain-untrue ($$70 <- missing) project: [$$53, $$61, $$70] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- STREAM_SELECT  |PARTITIONED|
                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                                  project ([$$53, $$61, $$70]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    select (eq($$55, $$71.getField(1))) retain-untrue ($$70 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- STREAM_SELECT  |PARTITIONED|
-                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        left-outer-unnest-map [$$70, $$71] <- index-search("ds_inner_typed", 0, "Default", "test", "ds_inner_typed", true, false, 1, $$61, 1, $$61, true, true, true) [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|
-                                            project ([$$53, $$55, $$61]) [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|
-                                                split ($$62) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- SPLIT  |PARTITIONED|
-                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    left-outer-unnest-map [$$60, $$61, $$62] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_typed", true, true, 1, $$55, 1, $$55, true, true, true) [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]
-                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                        project ([$$53, $$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          assign [$$55] <- [int32($$t1.getField(1))] [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]
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              data-scan []<-[$$53, $$t1] <- test.ds_outer_typed [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  |PARTITIONED|
-                              project ([$$61, $$53]) [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]
-                                -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                                  project ([$$53, $$61]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    select (eq($$55, $$60)) retain-untrue ($$61 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- STREAM_SELECT  |PARTITIONED|
-                                      project ([$$53, $$55, $$60, $$61]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  left-outer-unnest-map [$$70, $$71] <- index-search("ds_inner_typed", 0, "Default", "test", "ds_inner_typed", true, false, 1, $$61, 1, $$61, true, true, true) [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|
+                                      project ([$$53, $$55, $$61]) [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|
@@ -88,15 +48,43 @@
                                               -- BTREE_SEARCH  |PARTITIONED|
                                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                                 -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                  project ([$$53, $$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    assign [$$55] <- [int32($$t1.getField(1))] [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]
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        data-scan []<-[$$53, $$t1] <- test.ds_outer_typed [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|
+                                                  assign [$$55] <- [int32($$t1.getField(1))] project: [$$53, $$55] [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]
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      data-scan []<-[$$53, $$t1] <- test.ds_outer_typed [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  |PARTITIONED|
+                          project ([$$61, $$53]) [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]
+                            -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                              select (eq($$55, $$60)) retain-untrue ($$61 <- missing) project: [$$53, $$61] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- STREAM_SELECT  |PARTITIONED|
+                                project ([$$53, $$55, $$60, $$61]) [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|
+                                    split ($$62) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- SPLIT  |PARTITIONED|
+                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        left-outer-unnest-map [$$60, $$61, $$62] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_typed", true, true, 1, $$55, 1, $$55, true, true, true) [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]
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            assign [$$55] <- [int32($$t1.getField(1))] project: [$$53, $$55] [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]
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                data-scan []<-[$$53, $$t1] <- test.ds_outer_typed [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_less_parallelism/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.008.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_less_parallelism/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.008.plan
index 07a32cf..88abba5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_less_parallelism/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.008.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_less_parallelism/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.008.plan
@@ -2,81 +2,41 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$52] <- [{"t1_id": $$73, "t2_id": $$54}] [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 [$$73(ASC), $$54(ASC) ]  |PARTITIONED|
-          order (ASC, $$73) (ASC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$73(ASC), $$54(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$73, $$54]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select ($$74) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  window-aggregate [$$74] <- [win-mark-first-missing-impl($$54)] partition [$$73] order (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- WINDOW_STREAM  |PARTITIONED|
+    assign [$$52] <- [{"t1_id": $$73, "t2_id": $$54}] project: [$$52] [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 [$$73(ASC), $$54(ASC) ]  |PARTITIONED|
+        order (ASC, $$73) (ASC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$73(ASC), $$54(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select ($$74) project: [$$73, $$54] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              window-aggregate [$$74] <- [win-mark-first-missing-impl($$54)] partition [$$73] order (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- WINDOW_STREAM  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$73) (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STABLE_SORT [$$73(ASC), $$54(DESC)]  |PARTITIONED|
                     exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      order (ASC, $$73) (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STABLE_SORT [$$73(ASC), $$54(DESC)]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$73]  |PARTITIONED|
+                      union ($$70, $$61, $$54) ($$53, $$53, $$73) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- UNION_ALL  |PARTITIONED|
                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HASH_PARTITION_EXCHANGE [$$73]  |PARTITIONED|
-                          union ($$70, $$61, $$54) ($$53, $$53, $$73) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- UNION_ALL  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          project ([$$70, $$53]) [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|
-                              project ([$$70, $$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
+                            -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                              select (eq($$55, $$71.getField(1))) retain-untrue ($$70 <- missing) project: [$$53, $$61, $$70] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- STREAM_SELECT  |PARTITIONED|
                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                                  project ([$$53, $$61, $$70]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    select (eq($$55, $$71.getField(1))) retain-untrue ($$70 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- STREAM_SELECT  |PARTITIONED|
-                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        left-outer-unnest-map [$$70, $$71] <- index-search("ds_inner_typed", 0, "Default", "test", "ds_inner_typed", true, false, 1, $$61, 1, $$61, true, true, true) [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|
-                                            project ([$$53, $$55, $$61]) [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|
-                                                split ($$62) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- SPLIT  |PARTITIONED|
-                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    left-outer-unnest-map [$$60, $$61, $$62] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_typed", true, true, 1, $$55, 1, $$55, true, true, true) [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]
-                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                        project ([$$53, $$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          assign [$$55] <- [int32($$t1.getField("c_int32"))] [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]
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped [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  |PARTITIONED|
-                              project ([$$61, $$53]) [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]
-                                -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                                  project ([$$53, $$61]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    select (eq($$55, $$60)) retain-untrue ($$61 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- STREAM_SELECT  |PARTITIONED|
-                                      project ([$$53, $$55, $$60, $$61]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  left-outer-unnest-map [$$70, $$71] <- index-search("ds_inner_typed", 0, "Default", "test", "ds_inner_typed", true, false, 1, $$61, 1, $$61, true, true, true) [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|
+                                      project ([$$53, $$55, $$61]) [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|
@@ -88,15 +48,43 @@
                                               -- BTREE_SEARCH  |PARTITIONED|
                                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                                 -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                  project ([$$53, $$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    assign [$$55] <- [int32($$t1.getField("c_int32"))] [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]
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped [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|
+                                                  assign [$$55] <- [int32($$t1.getField("c_int32"))] project: [$$53, $$55] [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]
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped [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  |PARTITIONED|
+                          project ([$$61, $$53]) [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]
+                            -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                              select (eq($$55, $$60)) retain-untrue ($$61 <- missing) project: [$$53, $$61] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- STREAM_SELECT  |PARTITIONED|
+                                project ([$$53, $$55, $$60, $$61]) [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|
+                                    split ($$62) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- SPLIT  |PARTITIONED|
+                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        left-outer-unnest-map [$$60, $$61, $$62] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_typed", true, true, 1, $$55, 1, $$55, true, true, true) [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]
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            assign [$$55] <- [int32($$t1.getField("c_int32"))] project: [$$53, $$55] [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]
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                data-scan []<-[$$53, $$t1] <- test.ds_outer_untyped [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_less_parallelism/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.010.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results_less_parallelism/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.010.plan
index 2845b04..fe416df 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_less_parallelism/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.010.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_less_parallelism/leftouterjoin/index-only-leftouterjoin/index-only-leftouterjoin.010.plan
@@ -2,81 +2,41 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$52]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$52] <- [{"t1_id": $$73, "t2_id": $$54}] [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 [$$73(ASC), $$54(ASC) ]  |PARTITIONED|
-          order (ASC, $$73) (ASC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-          -- STABLE_SORT [$$73(ASC), $$54(ASC)]  |PARTITIONED|
-            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$73, $$54]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                select ($$74) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- STREAM_SELECT  |PARTITIONED|
-                  window-aggregate [$$74] <- [win-mark-first-missing-impl($$54)] partition [$$73] order (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                  -- WINDOW_STREAM  |PARTITIONED|
+    assign [$$52] <- [{"t1_id": $$73, "t2_id": $$54}] project: [$$52] [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 [$$73(ASC), $$54(ASC) ]  |PARTITIONED|
+        order (ASC, $$73) (ASC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$73(ASC), $$54(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            select ($$74) project: [$$73, $$54] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              window-aggregate [$$74] <- [win-mark-first-missing-impl($$54)] partition [$$73] order (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- WINDOW_STREAM  |PARTITIONED|
+                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$73) (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- STABLE_SORT [$$73(ASC), $$54(DESC)]  |PARTITIONED|
                     exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      order (ASC, $$73) (DESC, $$54) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- STABLE_SORT [$$73(ASC), $$54(DESC)]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$73]  |PARTITIONED|
+                      union ($$70, $$61, $$54) ($$53, $$53, $$73) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- UNION_ALL  |PARTITIONED|
                         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- HASH_PARTITION_EXCHANGE [$$73]  |PARTITIONED|
-                          union ($$70, $$61, $$54) ($$53, $$53, $$73) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                          -- UNION_ALL  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          project ([$$70, $$53]) [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|
-                              project ([$$70, $$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                              -- STREAM_PROJECT  |PARTITIONED|
+                            -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                              select (eq($$55, $$71.getField(1))) retain-untrue ($$70 <- missing) project: [$$53, $$61, $$70] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- STREAM_SELECT  |PARTITIONED|
                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                                  project ([$$53, $$61, $$70]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    select (eq($$55, $$71.getField(1))) retain-untrue ($$70 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- STREAM_SELECT  |PARTITIONED|
-                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        left-outer-unnest-map [$$70, $$71] <- index-search("ds_inner_typed", 0, "Default", "test", "ds_inner_typed", true, false, 1, $$61, 1, $$61, true, true, true) [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|
-                                            project ([$$53, $$55, $$61]) [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|
-                                                split ($$62) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                -- SPLIT  |PARTITIONED|
-                                                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    left-outer-unnest-map [$$60, $$61, $$62] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_typed", true, true, 1, $$55, 1, $$55, true, true, true) [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]
-                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                        project ([$$53, $$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          assign [$$55] <- [int32($$t1.getField(1))] [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]
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              data-scan []<-[$$53, $$t1] <- test.ds_outer_typed [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  |PARTITIONED|
-                              project ([$$61, $$53]) [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]
-                                -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                                  project ([$$53, $$61]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    select (eq($$55, $$60)) retain-untrue ($$61 <- missing) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                    -- STREAM_SELECT  |PARTITIONED|
-                                      project ([$$53, $$55, $$60, $$61]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  left-outer-unnest-map [$$70, $$71] <- index-search("ds_inner_typed", 0, "Default", "test", "ds_inner_typed", true, false, 1, $$61, 1, $$61, true, true, true) [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|
+                                      project ([$$53, $$55, $$61]) [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|
@@ -88,15 +48,43 @@
                                               -- BTREE_SEARCH  |PARTITIONED|
                                                 exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                                                 -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                  project ([$$53, $$55]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    assign [$$55] <- [int32($$t1.getField(1))] [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]
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        data-scan []<-[$$53, $$t1] <- test.ds_outer_typed [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|
+                                                  assign [$$55] <- [int32($$t1.getField(1))] project: [$$53, $$55] [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]
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      data-scan []<-[$$53, $$t1] <- test.ds_outer_typed [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  |PARTITIONED|
+                          project ([$$61, $$53]) [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]
+                            -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                              select (eq($$55, $$60)) retain-untrue ($$61 <- missing) project: [$$53, $$61] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- STREAM_SELECT  |PARTITIONED|
+                                project ([$$53, $$55, $$60, $$61]) [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|
+                                    split ($$62) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                    -- SPLIT  |PARTITIONED|
+                                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        left-outer-unnest-map [$$60, $$61, $$62] <- index-search("idx_c_int32", 0, "Default", "test", "ds_inner_typed", true, true, 1, $$55, 1, $$55, true, true, true) [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]
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            assign [$$55] <- [int32($$t1.getField(1))] project: [$$53, $$55] [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]
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                data-scan []<-[$$53, $$t1] <- test.ds_outer_typed [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_parser_sqlpp/string/like_01/like_01.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/like_01/like_01.3.ast
index d5c1517..c9e6664 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/like_01/like_01.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/like_01/like_01.3.ast
@@ -10,13 +10,7 @@
     LiteralExpr [STRING] [A8BBB]
     LiteralExpr [STRING] [_6%]
   ]
-  FunctionCall asterix.like@2[
-    LiteralExpr [STRING] [+0300]
-    LiteralExpr [STRING] [+03%]
-  ]
-  FunctionCall asterix.like@2[
-    LiteralExpr [STRING] [?0300]
-    LiteralExpr [STRING] [?03%]
-  ]
+  LiteralExpr [STRING] [+0300]
+  LiteralExpr [STRING] [?0300]
 ]
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/sqlpp_queries.xml b/asterixdb/asterix-app/src/test/resources/runtimets/sqlpp_queries.xml
index add376a..4b8d802 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/sqlpp_queries.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/sqlpp_queries.xml
@@ -631,6 +631,36 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="aggregate">
+      <compilation-unit name="countn_01">
+        <output-dir compare="Text">countn_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="countn_02">
+        <output-dir compare="Text">countn_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="countn_empty_01">
+        <output-dir compare="Text">countn_empty_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="countn_empty_02">
+        <output-dir compare="Text">countn_empty_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="countn_null">
+        <output-dir compare="Text">countn_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="countn_distinct">
+        <output-dir compare="Text">countn_distinct</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
       <compilation-unit name="kurtosis_double">
         <output-dir compare="Text">kurtosis_double</output-dir>
       </compilation-unit>
@@ -766,6 +796,21 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_countn">
+        <output-dir compare="Text">scalar_countn</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_countn_empty">
+        <output-dir compare="Text">scalar_countn_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_countn_null">
+        <output-dir compare="Text">scalar_countn_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
       <compilation-unit name="scalar_kurtosis">
         <output-dir compare="Text">scalar_kurtosis</output-dir>
       </compilation-unit>
@@ -1909,6 +1954,36 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="aggregate-sql">
+      <compilation-unit name="countn_01">
+        <output-dir compare="Text">countn_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="countn_02">
+        <output-dir compare="Text">countn_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="countn_empty_01">
+        <output-dir compare="Text">countn_empty_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="countn_empty_02">
+        <output-dir compare="Text">countn_empty_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="countn_null">
+        <output-dir compare="Text">countn_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="countn_distinct">
+        <output-dir compare="Text">countn_distinct</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
       <compilation-unit name="kurtosis_double">
         <output-dir compare="Text">kurtosis_double</output-dir>
       </compilation-unit>
@@ -2044,6 +2119,21 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_countn">
+        <output-dir compare="Text">scalar_countn</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_countn_empty">
+        <output-dir compare="Text">scalar_countn_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_countn_null">
+        <output-dir compare="Text">scalar_countn_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
       <compilation-unit name="scalar_kurtosis">
         <output-dir compare="Text">scalar_kurtosis</output-dir>
       </compilation-unit>
@@ -3046,6 +3136,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="array_fun">
+      <compilation-unit name="array_slice/array_slice_bracket_notation">
+        <output-dir compare="Text">array_slice/array_slice_bracket_notation</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
       <compilation-unit name="array_slice/array_slice_double_argument">
         <output-dir compare="Text">array_slice/array_slice_double_argument</output-dir>
       </compilation-unit>
@@ -6271,6 +6366,11 @@
         <expected-warn><![CDATA[ASX1132: Invalid specification for hint indexnl. ASX1001: Syntax error: In line 1 >>(8, idx_tenk2_1k_2k)<< Encountered <INTEGER_LITERAL> "8" at column 2.  (in line 35, at column 21)]]></expected-warn>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="index-join" check-warnings="true">
+      <compilation-unit name="secondary-heterogeneous-indexnl-params">
+        <output-dir compare="Text">secondary-heterogeneous-indexnl-params</output-dir>
+      </compilation-unit>
+    </test-case>
     <test-case FilePath="index-join">
       <compilation-unit name="rtree-spatial-intersect-point_01">
         <output-dir compare="Text">rtree-spatial-intersect-point_01</output-dir>
@@ -6587,10 +6687,37 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="index-selection">
+      <compilation-unit name="like-expression">
+        <output-dir compare="Text">like-expression</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
       <compilation-unit name="verify">
         <output-dir compare="Text">verify</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="heterogeneous-index-select">
+        <output-dir compare="Text">heterogeneous-index-select</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="heterogeneous-index-type-and-heterogeneous-key">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1079: Compilation error: Typed keys cannot be combined with untyped keys in the index definition. (in line 29, at column 1)</expected-error>
+        <expected-error>ASX1079: Compilation error: Typed keys cannot be combined with untyped keys in the index definition. (in line 29, at column 1)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="heterogeneous-index-bulkLoad">
+        <output-dir compare="Text">heterogeneous-index-bulkLoad</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="heterogeneous-index-Leftouterjoin">
+        <output-dir compare="Text">heterogeneous-index-Leftouterjoin</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="inverted-index-join">
     <test-case FilePath="inverted-index-join">
@@ -7436,6 +7563,11 @@
         <output-dir compare="Text">remove_listify</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="query-ASTERIXDB-3410">
+        <output-dir compare="Text">query-ASTERIXDB-3410</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="multipart-dataverse">
     <test-case FilePath="multipart-dataverse">
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_cloud_storage.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_cloud_storage.xml
index 6a75394..03fa519 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_cloud_storage.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_cloud_storage.xml
@@ -44,5 +44,65 @@
       </compilation-unit>
     </test-case>
   </test-group>
+  <test-group name="storage-size">
+    <test-case FilePath="cloud_storage/storage-size/datasource-function">
+      <compilation-unit name="collection-exists">
+        <output-dir compare="Clean-JSON">collection-exists</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cloud_storage/storage-size/datasource-function">
+      <compilation-unit name="non-constant-argument">
+        <output-dir compare="Clean-JSON">non-constant-argument</output-dir>
+        <expected-error>Compilation error: Function asterix.storage-size expects constant arguments while arg[0] is of type VARIABLE</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cloud_storage/storage-size/datasource-function">
+      <compilation-unit name="collection-does-not-exist">
+        <output-dir compare="Clean-JSON">collection-does-not-exist</output-dir>
+        <expected-error>Cannot find dataset with name doesNotExistCollection in dataverse testDatabase.testScope</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cloud_storage/storage-size/datasource-function">
+      <compilation-unit name="non-string-argument">
+        <output-dir compare="Clean-JSON">non-string-argument</output-dir>
+        <expected-error>Type mismatch: function storage-size expects its 1st input parameter to be of type string, but the actual input type is bigint</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cloud_storage/storage-size/datasource-function">
+      <compilation-unit name="view-not-collection">
+        <output-dir compare="Clean-JSON">view-not-collection</output-dir>
+        <expected-error>Cannot find dataset with name testCollection_vw in dataverse testDatabase.testScope</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cloud_storage/storage-size/datasource-function">
+      <compilation-unit name="external-collection">
+        <output-dir compare="Clean-JSON">external-collection</output-dir>
+        <expected-error>Retrieving storage size is not applicable to type: EXTERNAL.</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cloud_storage/storage-size/datasource-function">
+      <compilation-unit name="index-exists">
+        <output-dir compare="Clean-JSON">index-exists</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cloud_storage/storage-size/datasource-function">
+      <compilation-unit name="index-does-not-exist">
+        <output-dir compare="Clean-JSON">index-does-not-exist</output-dir>
+        <expected-error>Cannot find index with name testCollection_idx_fake</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cloud_storage/storage-size/datasource-function">
+      <compilation-unit name="invalid-arguments-count">
+        <output-dir compare="Clean-JSON">invalid-arguments-count</output-dir>
+        <expected-error>Invalid number of arguments for function storage-size</expected-error>
+        <expected-error>Invalid number of arguments for function storage-size</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cloud_storage/storage-size/datasource-function">
+      <compilation-unit name="sanity">
+        <output-dir compare="Clean-JSON">sanity</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
   &sqlpp_queries;
 </test-suite>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_azure_blob_storage.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_azure_blob_storage.xml
index ca0167b..b801847 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_azure_blob_storage.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_azure_blob_storage.xml
@@ -402,7 +402,7 @@
         <placeholder name="adapter" value="AZUREBLOB" />
         <placeholder name="endpoint" value="http://^invalid-endpoint^" />
         <output-dir compare="Text">common/invalid-endpoint</output-dir>
-        <expected-error>Illegal character in authority at index 7: http://^invalid-endpoint^</expected-error>
+        <expected-error>The Azure Storage endpoint url is malformed.</expected-error>
       </compilation-unit>
     </test-case>
     <!--
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_s3.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_s3.xml
index 8a76970..8a5151c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_s3.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_s3.xml
@@ -21,11 +21,21 @@
   <test-group name="copy-to">
     <test-case FilePath="copy-to">
       <compilation-unit name="partition">
+        <placeholder name="adapter" value="S3" />
+        <placeholder name="pathprefix" value="" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additionalProperties" value='"container":"playground",' />
+        <placeholder name="additional_Properties" value='("container"="playground")' />
         <output-dir compare="Text">partition</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="copy-to">
       <compilation-unit name="simple-write">
+        <placeholder name="adapter" value="S3" />
+        <placeholder name="pathprefix" value="" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additionalProperties" value='"container":"playground",' />
+        <placeholder name="additional_Properties" value='("container"="playground")' />
         <output-dir compare="Text">simple-write</output-dir>
       </compilation-unit>
     </test-case>
@@ -36,21 +46,105 @@
     </test-case>
     <test-case FilePath="copy-to">
       <compilation-unit name="default-namespace">
+        <placeholder name="adapter" value="S3" />
+        <placeholder name="pathprefix" value="" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additionalProperties" value='"container":"playground",' />
+        <placeholder name="additional_Properties" value='("container"="playground")' />
         <output-dir compare="Text">default-namespace</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="copy-to">
+      <compilation-unit name="parquet-simple">
+        <placeholder name="adapter" value="S3" />
+        <placeholder name="pathprefix" value="" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additionalProperties" value='"container":"playground",' />
+        <placeholder name="additional_Properties" value='("container"="playground")' />
+        <output-dir compare="Text">parquet-simple</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to">
+      <compilation-unit name="parquet-tweet">
+        <placeholder name="adapter" value="S3" />
+        <placeholder name="pathprefix" value="" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additionalProperties" value='"container":"playground",' />
+        <placeholder name="additional_Properties" value='("container"="playground")' />
+        <output-dir compare="Text">parquet-tweet</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to">
+      <compilation-unit name="parquet-partition-heterogeneous">
+        <placeholder name="adapter" value="S3" />
+        <placeholder name="pathprefix" value="" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additionalProperties" value='"container":"playground",' />
+        <placeholder name="additional_Properties" value='("container"="playground")' />
+        <output-dir compare="Text">parquet-partition-heterogeneous</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to">
+      <compilation-unit name="parquet-utf8">
+        <placeholder name="adapter" value="S3" />
+        <placeholder name="pathprefix" value="" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additionalProperties" value='"container":"playground",' />
+        <placeholder name="additional_Properties" value='("container"="playground")' />
+        <output-dir compare="Text">parquet-utf8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to">
+      <compilation-unit name="parquet-heterogeneous">
+        <placeholder name="adapter" value="S3" />
+        <placeholder name="pathprefix" value="" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additionalProperties" value='"container":"playground",' />
+        <placeholder name="additional_Properties" value='("container"="playground")' />
+        <output-dir compare="Text">parquet-heterogeneous</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to">
+      <compilation-unit name="parquet-cover-data-types">
+        <placeholder name="adapter" value="S3" />
+        <placeholder name="pathprefix" value="" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additionalProperties" value='"container":"playground",' />
+        <placeholder name="additional_Properties" value='("container"="playground")' />
+        <output-dir compare="Text">parquet-cover-data-types</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to">
+      <compilation-unit name="parquet-empty-array">
+        <placeholder name="adapter" value="S3" />
+        <placeholder name="pathprefix" value="" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additionalProperties" value='"container":"playground",' />
+        <placeholder name="additional_Properties" value='("container"="playground")' />
+        <output-dir compare="Text">parquet-empty-array</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to">
       <compilation-unit name="empty-path">
         <output-dir compare="Text">empty-path</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="copy-to">
       <compilation-unit name="order-by">
+        <placeholder name="adapter" value="S3" />
+        <placeholder name="pathprefix" value="" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additionalProperties" value='"container":"playground",' />
+        <placeholder name="additional_Properties" value='("container"="playground")' />
         <output-dir compare="Text">order-by</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="copy-to/negative">
       <compilation-unit name="early-missing">
+        <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additionalProperties" value='"container":"playground",' />
+        <placeholder name="additional_Properties" value='("container"="playground")' />
         <output-dir compare="Text">early-missing</output-dir>
         <expected-error>ASX0064: Path expression produced a value of type 'missing'. Path must be of type string</expected-error>
         <expected-error>ASX0064: Path expression produced a value of type 'null'. Path must be of type string</expected-error>
@@ -66,6 +160,11 @@
     <test-case FilePath="copy-to/negative">
       <compilation-unit name="non-empty-folder">
         <output-dir compare="Text">non-empty-folder</output-dir>
+        <placeholder name="adapter" value="S3" />
+        <placeholder name="pathprefix" value="" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additionalProperties" value='"container":"playground",' />
+        <placeholder name="additional_Properties" value='("container"="playground")' />
         <expected-error>ASX0062: Cannot write to a non-empty directory 'copy-to-result/duplicate-write'</expected-error>
       </compilation-unit>
     </test-case>
@@ -78,25 +177,135 @@
     </test-case>
     <test-case FilePath="copy-to/negative">
       <compilation-unit name="supported-adapter-format-compression">
+        <placeholder name="adapter" value="S3" />
+        <placeholder name="pathprefix" value="" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additionalProperties" value='"container":"playground",' />
+        <placeholder name="additional_Properties" value='("container"="playground")' />
         <output-dir compare="Text">supported-adapter-format-compression</output-dir>
-        <expected-error>ASX1188: Unsupported writing adapter 'AZUREBLOB'. Supported adapters: [gcs, localfs, s3]</expected-error>
-        <expected-error>ASX1189: Unsupported writing format 'csv'. Supported formats: [json]</expected-error>
-        <expected-error>ASX1096: Unknown compression scheme rar. Supported schemes are [gzip]</expected-error>
+        <expected-error>ASX1188: Unsupported writing adapter 'AZUREBLOB'. Supported adapters: [gcs, hdfs, localfs, s3]</expected-error>
+        <expected-error>ASX1189: Unsupported writing format 'avro'. Supported formats: [csv, json, parquet]</expected-error>
+        <expected-error>ASX1202: Unsupported compression scheme rar. Supported schemes for json are [gzip]</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to/negative">
+      <compilation-unit name="parquet-error-checks">
+        <placeholder name="adapter" value="S3" />
+        <placeholder name="pathprefix" value="" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additionalProperties" value='"container":"playground",' />
+        <placeholder name="additional_Properties" value='("container"="playground")' />
+        <output-dir compare="Text">parquet-error-checks</output-dir>
+        <expected-error>ASX0037: Type mismatch: expected value of type BINARY, but got the value of type bigint</expected-error>
+        <expected-error>HYR0132: Extra field in the result, field 'second' does not exist at 'nested' in the schema</expected-error>
+        <expected-error>HYR0131: Result does not follow the schema, group type expected but found primitive type at 'nested'</expected-error>
+        <expected-error>HYR0131: Result does not follow the schema, primitive type expected but found group type at 'name'</expected-error>
+        <expected-error>ASX1206: Storage units expected for the field 'row-group-size' (e.g., 0.1KB, 100kb, 1mb, 3MB, 8.5GB ...). Provided 'random'</expected-error>
+        <expected-error>ASX1206: Storage units expected for the field 'page-size' (e.g., 0.1KB, 100kb, 1mb, 3MB, 8.5GB ...). Provided 'random'</expected-error>
+        <expected-error>ASX1202: Unsupported compression scheme rar. Supported schemes for parquet are [gzip, snappy, zstd]</expected-error>
+        <expected-error>ASX1001: Syntax error</expected-error>
+        <expected-error>ASX1204: 'binary' type not supported in parquet format</expected-error>
+        <expected-error>ASX1205: Invalid Parquet Writer Version provided '3'. Supported values: [1, 2]</expected-error>
+        <expected-error>ASX0039: Expected integer value, got yvghc (in line 22, at column 6)</expected-error>
+        <expected-error>ASX1209: Maximum value allowed for 'max-schemas' is 10. Found 15</expected-error>
+        <expected-error>HYR0133: Schema could not be inferred, empty types found in the result</expected-error>
+        <expected-error>HYR0134: Schema Limit exceeded, maximum number of heterogeneous schemas allowed : '2'</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="copy-to/negative">
       <compilation-unit name="empty-over">
+        <placeholder name="adapter" value="S3" />
+        <placeholder name="pathprefix" value="" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additionalProperties" value='"container":"playground",' />
+        <placeholder name="additional_Properties" value='("container"="playground")' />
         <output-dir compare="Text">empty-over</output-dir>
         <expected-error>ASX1001: Syntax error: OVER-clause cannot be empty</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="copy-to/negative">
       <compilation-unit name="bad-max-objects-per-file">
+        <placeholder name="adapter" value="S3" />
+        <placeholder name="pathprefix" value="" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additionalProperties" value='"container":"playground",' />
+        <placeholder name="additional_Properties" value='("container"="playground")' />
         <output-dir compare="Text">bad-max-objects-per-file</output-dir>
         <expected-error>Minimum value allowed for 'max-objects-per-file' is 1000. Found 2</expected-error>
         <expected-error>Expected integer value, got hello</expected-error>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="copy-to/negative">
+      <compilation-unit name="csv-error-checks">
+        <placeholder name="adapter" value="S3" />
+        <placeholder name="pathprefix" value="" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additionalProperties" value='"container":"playground",' />
+        <placeholder name="additional_Properties" value='("container"="playground")' />
+        <output-dir compare="Text">csv-error-checks</output-dir>
+        <expected-error>ASX1079: Compilation error: TYPE/AS Expression is required for csv format</expected-error>
+        <expected-error>ASX1082: Cannot find datatype with name wrongDataType (in line 27, at column 4)</expected-error>
+        <expected-error>ASX3124: 'ABCD' is not a valid quote. The length of a quote should be 1</expected-error>
+        <expected-error>ASX3049: 'wrongDelimiter' is not a valid delimiter. The length of a delimiter should be 1</expected-error>
+        <expected-error>ASX3126: 'wrongEscape' is not a valid escape. The length of a escape should be 1</expected-error>
+        <expected-error>ASX3125: 'ABCD' is not a valid force-quote input. The length of a force-quote input should be 1 character</expected-error>
+        <expected-error>ASX1207: 'object' type not supported in csv format</expected-error>
+        <expected-error>ASX1207: 'array' type not supported in csv format</expected-error>
+        <expected-error>Syntax error: Both 'TYPE()' and 'AS()' are provided. Please use either 'TYPE()' or 'AS()'.</expected-error>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="copy-to/csv">
+    <test-case FilePath="copy-to/csv">
+      <compilation-unit name="simple-csv">
+        <placeholder name="adapter" value="S3" />
+        <placeholder name="pathprefix" value="" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additionalProperties" value='"container":"playground",' />
+        <placeholder name="additional_Properties" value='("container"="playground")' />
+        <output-dir compare="Text">simple-csv</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to/csv">
+      <compilation-unit name="type-mismatch">
+        <placeholder name="adapter" value="S3" />
+        <placeholder name="pathprefix" value="" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additionalProperties" value='"container":"playground",' />
+        <placeholder name="additional_Properties" value='("container"="playground")' />
+        <output-dir compare="Text">type-mismatch</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to/csv">
+      <compilation-unit name="delimiter">
+        <placeholder name="adapter" value="S3" />
+        <placeholder name="pathprefix" value="" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additionalProperties" value='"container":"playground",' />
+        <placeholder name="additional_Properties" value='("container"="playground")' />
+        <output-dir compare="Text">delimiter</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to/csv">
+      <compilation-unit name="header">
+        <placeholder name="adapter" value="S3" />
+        <placeholder name="pathprefix" value="" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additionalProperties" value='"container":"playground",' />
+        <placeholder name="additional_Properties" value='("container"="playground")' />
+        <output-dir compare="Text">header</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to/csv">
+      <compilation-unit name="quote-escape">
+        <placeholder name="adapter" value="S3" />
+        <placeholder name="pathprefix" value="" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additionalProperties" value='"container":"playground",' />
+        <placeholder name="additional_Properties" value='("container"="playground")' />
+        <output-dir compare="Text">quote-escape</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="aws-s3-external-dataset">
     <test-case FilePath="external-dataset">
@@ -128,16 +337,16 @@
         <output-dir compare="Text">common/csv/csv</output-dir>
       </compilation-unit>
     </test-case><test-case FilePath="external-dataset">
-      <compilation-unit name="common/csv/gz">
-        <placeholder name="adapter" value="S3" />
-        <output-dir compare="Text">common/csv/gz</output-dir>
-      </compilation-unit>
-    </test-case><test-case FilePath="external-dataset">
-      <compilation-unit name="common/csv/mixed">
-        <placeholder name="adapter" value="S3" />
-        <output-dir compare="Text">common/csv/mixed</output-dir>
-      </compilation-unit>
-    </test-case>
+    <compilation-unit name="common/csv/gz">
+      <placeholder name="adapter" value="S3" />
+      <output-dir compare="Text">common/csv/gz</output-dir>
+    </compilation-unit>
+  </test-case><test-case FilePath="external-dataset">
+    <compilation-unit name="common/csv/mixed">
+      <placeholder name="adapter" value="S3" />
+      <output-dir compare="Text">common/csv/mixed</output-dir>
+    </compilation-unit>
+  </test-case>
     <test-case FilePath="external-dataset">
       <compilation-unit name="common/tsv/tsv">
         <placeholder name="adapter" value="S3" />
@@ -353,74 +562,110 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-dataset">
+      <compilation-unit name="common/deltalake-file-one">
+        <placeholder name="adapter" value="S3" />
+        <output-dir compare="Text">common/deltalake-file-one</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/deltalake-file-nine">
+        <placeholder name="adapter" value="S3" />
+        <output-dir compare="Text">common/deltalake-file-nine</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
       <compilation-unit name="common/avro/avro-types/avro-map">
         <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additional_Properties" value='("container" = "playground")' />
         <output-dir compare="Text">common/avro/avro-types/avro-map</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-dataset">
       <compilation-unit name="common/avro/avro-types/avro-nested-records">
         <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additional_Properties" value='("container" = "playground")' />
         <output-dir compare="Text">common/avro/avro-types/avro-nested-records</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-dataset">
       <compilation-unit name="common/avro/avro-types/avro-primitives">
         <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additional_Properties" value='("container" = "playground")' />
         <output-dir compare="Text">common/avro/avro-types/avro-primitives</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-dataset">
       <compilation-unit name="common/avro/avro-types/avro-union">
         <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additional_Properties" value='("container" = "playground")' />
         <output-dir compare="Text">common/avro/avro-types/avro-union</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-dataset">
       <compilation-unit name="common/avro/type-mismatch">
         <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additional_Properties" value='("container" = "playground")' />
         <output-dir compare="Text">common/avro/type-mismatch</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-dataset">
       <compilation-unit name="common/avro/string-standard-utf8">
         <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additional_Properties" value='("container" = "playground")' />
         <output-dir compare="Text">common/avro/string-standard-utf8</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-dataset">
       <compilation-unit name="common/avro/select-all-fields">
         <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additional_Properties" value='("container" = "playground")' />
         <output-dir compare="Text">common/avro/select-all-fields</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-dataset">
       <compilation-unit name="common/avro/select-count-one-field">
         <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additional_Properties" value='("container" = "playground")' />
         <output-dir compare="Text">common/avro/select-count-one-field</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-dataset">
       <compilation-unit name="common/avro/array-access">
         <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additional_Properties" value='("container" = "playground")' />
         <output-dir compare="Text">common/avro/array-access</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-dataset">
       <compilation-unit name="common/avro/field-access">
         <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additional_Properties" value='("container" = "playground")' />
         <output-dir compare="Text">common/avro/field-access</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-dataset">
       <compilation-unit name="common/avro/heterogeneous-access">
         <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additional_Properties" value='("container" = "playground")' />
         <output-dir compare="Text">common/avro/heterogeneous-access</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-dataset">
       <compilation-unit name="common/avro/invalid-avro-files">
         <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additional_Properties" value='("container" = "playground")' />
         <output-dir compare="Text">none</output-dir>
         <expected-error>External source error. org.apache.avro.InvalidAvroMagicException: Not an Avro data file.</expected-error>
         <source-location>false</source-location>
@@ -429,6 +674,8 @@
     <test-case FilePath="external-dataset">
       <compilation-unit name="common/avro/invalid-type">
         <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additional_Properties" value='("container" = "playground")' />
         <output-dir compare="Text">none</output-dir>
         <expected-error>ASX3123: Type 'AvroType' contains declared fields, which is not supported for 'avro' format</expected-error>
       </compilation-unit>
@@ -436,24 +683,32 @@
     <test-case FilePath="external-dataset">
       <compilation-unit name="common/avro/missing-fields">
         <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additional_Properties" value='("container" = "playground")' />
         <output-dir compare="Text">common/avro/missing-fields</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-dataset">
       <compilation-unit name="common/avro/multi-file-multi-schema">
         <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additional_Properties" value='("container" = "playground")' />
         <output-dir compare="Text">common/avro/multi-file-multi-schema</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-dataset">
       <compilation-unit name="common/avro/no-files">
         <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additional_Properties" value='("container" = "playground")' />
         <output-dir compare="Text">common/avro/no-files</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-dataset">
       <compilation-unit name="common/avro/object-concat">
         <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additional_Properties" value='("container" = "playground")' />
         <output-dir compare="Text">common/avro/object-concat</output-dir>
       </compilation-unit>
     </test-case>
@@ -688,48 +943,64 @@
     <test-case FilePath="external-dataset/common/dynamic-prefixes/avro">
       <compilation-unit name="one-field">
         <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additional_Properties" value='("container" = "playground")' />
         <output-dir compare="Text">one-field</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-dataset/common/dynamic-prefixes/avro">
       <compilation-unit name="query">
         <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additional_Properties" value='("container" = "playground")' />
         <output-dir compare="Text">query</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-dataset/common/dynamic-prefixes/avro">
       <compilation-unit name="embed-one-value">
         <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additional_Properties" value='("container" = "playground")' />
         <output-dir compare="Text">embed-one-value</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-dataset/common/dynamic-prefixes/avro">
       <compilation-unit name="embed-multiple-values">
         <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additional_Properties" value='("container" = "playground")' />
         <output-dir compare="Text">embed-multiple-values</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-dataset/common/dynamic-prefixes/avro">
       <compilation-unit name="embed-flat">
         <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additional_Properties" value='("container" = "playground")' />
         <output-dir compare="Text">embed-flat</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-dataset/common/dynamic-prefixes/avro">
       <compilation-unit name="embed-with-nested-values">
         <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additional_Properties" value='("container" = "playground")' />
         <output-dir compare="Text">embed-with-nested-values</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-dataset/common/dynamic-prefixes/avro">
       <compilation-unit name="using-limit">
         <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additional_Properties" value='("container" = "playground")' />
         <output-dir compare="Text">using-limit</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-dataset/common/dynamic-prefixes/avro">
       <compilation-unit name="views">
         <placeholder name="adapter" value="S3" />
+        <placeholder name="path_prefix" value="" />
+        <placeholder name="additional_Properties" value='("container" = "playground")' />
         <output-dir compare="Text">views</output-dir>
       </compilation-unit>
     </test-case>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_hdfs.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_hdfs.xml
index 9e39211..b178efd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_hdfs.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_hdfs.xml
@@ -66,4 +66,505 @@
       </test-case>
       -->
   </test-group>
+  <test-group name="copy-to">
+    <test-case FilePath="copy-to-hdfs">
+      <compilation-unit name="parquet-simple">
+        <output-dir compare="Text">parquet-simple</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to-hdfs">
+      <compilation-unit name="parquet-tweet">
+        <output-dir compare="Text">parquet-tweet</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to-hdfs">
+      <compilation-unit name="parquet-utf8">
+        <output-dir compare="Text">parquet-utf8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to-hdfs">
+      <compilation-unit name="parquet-cover-data-types">
+        <output-dir compare="Text">parquet-cover-data-types</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to-hdfs">
+      <compilation-unit name="parquet-empty-array">
+        <output-dir compare="Text">parquet-empty-array</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to">
+      <compilation-unit name="partition">
+        <placeholder name="adapter" value="HDFS" />
+        <placeholder name="pathprefix" value='"/playground", ' />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additionalProperties" value="" />
+        <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+        <output-dir compare="Text">partition</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to">
+      <compilation-unit name="simple-write">
+        <placeholder name="adapter" value="HDFS" />
+        <placeholder name="pathprefix" value='"/playground", ' />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additionalProperties" value="" />
+        <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+        <output-dir compare="Text">simple-write</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to">
+      <compilation-unit name="default-namespace">
+        <placeholder name="adapter" value="HDFS" />
+        <placeholder name="pathprefix" value='"/playground", ' />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additionalProperties" value="" />
+        <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+        <output-dir compare="Text">default-namespace</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to">
+      <compilation-unit name="parquet-simple">
+        <placeholder name="adapter" value="HDFS" />
+        <placeholder name="pathprefix" value='"/playground", ' />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additionalProperties" value="" />
+        <placeholder name="additional_Properties" value='("input-format" = "parquet-input-format")' />
+        <output-dir compare="Text">parquet-simple</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to">
+      <compilation-unit name="parquet-tweet">
+        <placeholder name="adapter" value="HDFS" />
+        <placeholder name="pathprefix" value='"/playground", ' />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additionalProperties" value="" />
+        <placeholder name="additional_Properties" value='("input-format" = "parquet-input-format")' />
+        <output-dir compare="Text">parquet-tweet</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to">
+      <compilation-unit name="parquet-partition-heterogeneous">
+        <placeholder name="adapter" value="HDFS" />
+        <placeholder name="pathprefix" value='"/playground", ' />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additionalProperties" value="" />
+        <placeholder name="additional_Properties" value='("input-format" = "parquet-input-format")' />
+        <output-dir compare="Text">parquet-partition-heterogeneous</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to">
+      <compilation-unit name="parquet-utf8">
+        <placeholder name="adapter" value="HDFS" />
+        <placeholder name="pathprefix" value='"/playground", ' />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additionalProperties" value="" />
+        <placeholder name="additional_Properties" value='("input-format" = "parquet-input-format")' />
+        <output-dir compare="Text">parquet-utf8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to">
+      <compilation-unit name="parquet-heterogeneous">
+        <placeholder name="adapter" value="HDFS" />
+        <placeholder name="pathprefix" value='"/playground", ' />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additionalProperties" value="" />
+        <placeholder name="additional_Properties" value='("input-format" = "parquet-input-format")' />
+        <output-dir compare="Text">parquet-heterogeneous</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to">
+      <compilation-unit name="parquet-cover-data-types">
+        <placeholder name="adapter" value="HDFS" />
+        <placeholder name="pathprefix" value='"/playground", ' />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additionalProperties" value="" />
+        <placeholder name="additional_Properties" value='("input-format" = "parquet-input-format")' />
+        <output-dir compare="Text">parquet-cover-data-types</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to">
+      <compilation-unit name="parquet-empty-array">
+        <placeholder name="adapter" value="HDFS" />
+        <placeholder name="pathprefix" value='"/playground", ' />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additionalProperties" value="" />
+        <placeholder name="additional_Properties" value='("input-format" = "parquet-input-format")' />
+        <output-dir compare="Text">parquet-empty-array</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to">
+      <compilation-unit name="order-by">
+        <placeholder name="adapter" value="HDFS" />
+        <placeholder name="pathprefix" value='"/playground", ' />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additionalProperties" value="" />
+        <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+        <output-dir compare="Text">order-by</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to/negative">
+      <compilation-unit name="early-missing">
+        <placeholder name="adapter" value="HDFS" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additionalProperties" value="" />
+        <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+        <output-dir compare="Text">early-missing</output-dir>
+        <expected-error>ASX0064: Path expression produced a value of type 'missing'. Path must be of type string</expected-error>
+        <expected-error>ASX0064: Path expression produced a value of type 'null'. Path must be of type string</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to/negative">
+      <compilation-unit name="non-empty-folder">
+        <placeholder name="adapter" value="HDFS" />
+        <placeholder name="pathprefix" value='"/playground", ' />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additionalProperties" value="" />
+        <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+        <output-dir compare="Text">non-empty-folder</output-dir>
+        <expected-error>ASX0062: Cannot write to a non-empty directory '/playground/copy-to-result/duplicate-write'</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to/negative">
+      <compilation-unit name="supported-adapter-format-compression">
+        <placeholder name="adapter" value="HDFS" />
+        <placeholder name="pathprefix" value='"/playground", ' />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additionalProperties" value="" />
+        <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+        <output-dir compare="Text">supported-adapter-format-compression</output-dir>
+        <expected-error>ASX1188: Unsupported writing adapter 'AZUREBLOB'. Supported adapters: [gcs, hdfs, localfs, s3]</expected-error>
+        <expected-error>ASX1189: Unsupported writing format 'avro'. Supported formats: [csv, json, parquet]</expected-error>
+        <expected-error>ASX1202: Unsupported compression scheme rar. Supported schemes for json are [gzip]</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to/negative">
+      <compilation-unit name="parquet-error-checks">
+        <placeholder name="adapter" value="HDFS" />
+        <placeholder name="pathprefix" value='"/playground", ' />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additionalProperties" value="" />
+        <output-dir compare="Text">parquet-error-checks</output-dir>
+        <expected-error>ASX0037: Type mismatch: expected value of type BINARY, but got the value of type bigint</expected-error>
+        <expected-error>HYR0132: Extra field in the result, field 'second' does not exist at 'nested' in the schema</expected-error>
+        <expected-error>HYR0131: Result does not follow the schema, group type expected but found primitive type at 'nested'</expected-error>
+        <expected-error>HYR0131: Result does not follow the schema, primitive type expected but found group type at 'name'</expected-error>
+        <expected-error>ASX1206: Storage units expected for the field 'row-group-size' (e.g., 0.1KB, 100kb, 1mb, 3MB, 8.5GB ...). Provided 'random'</expected-error>
+        <expected-error>ASX1206: Storage units expected for the field 'page-size' (e.g., 0.1KB, 100kb, 1mb, 3MB, 8.5GB ...). Provided 'random'</expected-error>
+        <expected-error>ASX1202: Unsupported compression scheme rar. Supported schemes for parquet are [gzip, snappy, zstd]</expected-error>
+        <expected-error>ASX1001: Syntax error</expected-error>
+        <expected-error>ASX1204: 'binary' type not supported in parquet format</expected-error>
+        <expected-error>ASX1205: Invalid Parquet Writer Version provided '3'. Supported values: [1, 2]</expected-error>
+        <expected-error>ASX0039: Expected integer value, got yvghc (in line 22, at column 6)</expected-error>
+        <expected-error>ASX1209: Maximum value allowed for 'max-schemas' is 10. Found 15</expected-error>
+        <expected-error>HYR0133: Schema could not be inferred, empty types found in the result</expected-error>
+        <expected-error>HYR0134: Schema Limit exceeded, maximum number of heterogeneous schemas allowed : '2'</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to/negative">
+      <compilation-unit name="empty-over">
+        <placeholder name="adapter" value="HDFS" />
+        <placeholder name="pathprefix" value='"/playground", ' />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additionalProperties" value="" />
+        <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+        <output-dir compare="Text">empty-over</output-dir>
+        <expected-error>ASX1001: Syntax error: OVER-clause cannot be empty</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to/negative">
+      <compilation-unit name="bad-max-objects-per-file">
+        <placeholder name="adapter" value="HDFS" />
+        <placeholder name="pathprefix" value='"/playground", ' />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additionalProperties" value="" />
+        <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+        <output-dir compare="Text">bad-max-objects-per-file</output-dir>
+        <expected-error>Minimum value allowed for 'max-objects-per-file' is 1000. Found 2</expected-error>
+        <expected-error>Expected integer value, got hello</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to/negative">
+      <compilation-unit name="csv-error-checks">
+        <placeholder name="adapter" value="HDFS" />
+        <placeholder name="pathprefix" value='"/playground", ' />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additionalProperties" value="" />
+        <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+        <output-dir compare="Text">csv-error-checks</output-dir>
+        <expected-error>ASX1079: Compilation error: TYPE/AS Expression is required for csv format</expected-error>
+        <expected-error>ASX1082: Cannot find datatype with name wrongDataType (in line 27, at column 4)</expected-error>
+        <expected-error>ASX3124: 'ABCD' is not a valid quote. The length of a quote should be 1</expected-error>
+        <expected-error>ASX3049: 'wrongDelimiter' is not a valid delimiter. The length of a delimiter should be 1</expected-error>
+        <expected-error>ASX3126: 'wrongEscape' is not a valid escape. The length of a escape should be 1</expected-error>
+        <expected-error>ASX3125: 'ABCD' is not a valid force-quote input. The length of a force-quote input should be 1 character</expected-error>
+        <expected-error>ASX1207: 'object' type not supported in csv format</expected-error>
+        <expected-error>ASX1207: 'array' type not supported in csv format</expected-error>
+        <expected-error>Syntax error: Both 'TYPE()' and 'AS()' are provided. Please use either 'TYPE()' or 'AS()'.</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="copy-to/csv">
+    <test-case FilePath="copy-to/csv">
+      <compilation-unit name="simple-csv">
+        <placeholder name="adapter" value="HDFS" />
+        <placeholder name="pathprefix" value='"/playground", ' />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additionalProperties" value="" />
+        <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+        <output-dir compare="Text">simple-csv</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to/csv">
+      <compilation-unit name="type-mismatch">
+        <placeholder name="adapter" value="HDFS" />
+        <placeholder name="pathprefix" value='"/playground", ' />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additionalProperties" value="" />
+        <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+        <output-dir compare="Text">type-mismatch</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to/csv">
+      <compilation-unit name="delimiter">
+        <placeholder name="adapter" value="HDFS" />
+        <placeholder name="pathprefix" value='"/playground", ' />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additionalProperties" value="" />
+        <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+        <output-dir compare="Text">delimiter</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="copy-to/csv">
+      <compilation-unit name="header">
+        <placeholder name="adapter" value="HDFS" />
+        <placeholder name="pathprefix" value='"/playground", ' />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additionalProperties" value="" />
+        <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />
+        <output-dir compare="Text">header</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--    TODO: This test is failing: ASTERIXDB-3529-->
+    <!--    <test-case FilePath="copy-to/csv">-->
+    <!--      <compilation-unit name="quote-escape">-->
+    <!--        <placeholder name="adapter" value="HDFS" />-->
+    <!--        <placeholder name="pathprefix" value='"/playground", ' />-->
+    <!--        <placeholder name="path_prefix" value="/playground/" />-->
+    <!--        <placeholder name="additionalProperties" value="" />-->
+    <!--        <placeholder name="additional_Properties" value='("input-format" = "text-input-format")' />-->
+    <!--        <output-dir compare="Text">quote-escape</output-dir>-->
+    <!--      </compilation-unit>-->
+    <!--    </test-case>-->
+  </test-group>
+  <test-group name="avro">
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/avro-types/avro-map">
+        <placeholder name="adapter" value="hdfs" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additional_Properties" value='("input-format" = "avro-input-format")' />
+        <output-dir compare="Text">common/avro/avro-types/avro-map</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/avro-types/avro-nested-records">
+        <placeholder name="adapter" value="hdfs" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additional_Properties" value='("input-format" = "avro-input-format")' />
+        <output-dir compare="Text">common/avro/avro-types/avro-nested-records</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/avro-types/avro-primitives">
+        <placeholder name="adapter" value="hdfs" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additional_Properties" value='("input-format" = "avro-input-format")' />
+        <output-dir compare="Text">common/avro/avro-types/avro-primitives</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/avro-types/avro-union">
+        <placeholder name="adapter" value="hdfs" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additional_Properties" value='("input-format" = "avro-input-format")' />
+        <output-dir compare="Text">common/avro/avro-types/avro-union</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/type-mismatch">
+        <placeholder name="adapter" value="hdfs" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additional_Properties" value='("input-format" = "avro-input-format")' />
+        <output-dir compare="Text">common/avro/type-mismatch</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/string-standard-utf8">
+        <placeholder name="adapter" value="hdfs" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additional_Properties" value='("input-format" = "avro-input-format")' />
+        <output-dir compare="Text">common/avro/string-standard-utf8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/select-all-fields">
+        <placeholder name="adapter" value="hdfs" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additional_Properties" value='("input-format" = "avro-input-format")' />
+        <output-dir compare="Text">common/avro/select-all-fields</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/select-count-one-field">
+        <placeholder name="adapter" value="hdfs" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additional_Properties" value='("input-format" = "avro-input-format")' />
+        <output-dir compare="Text">common/avro/select-count-one-field</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/array-access">
+        <placeholder name="adapter" value="hdfs" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additional_Properties" value='("input-format" = "avro-input-format")' />
+        <output-dir compare="Text">common/avro/array-access</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/field-access">
+        <placeholder name="adapter" value="hdfs" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additional_Properties" value='("input-format" = "avro-input-format")' />
+        <output-dir compare="Text">common/avro/field-access</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/heterogeneous-access">
+        <placeholder name="adapter" value="hdfs" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additional_Properties" value='("input-format" = "avro-input-format")' />
+        <output-dir compare="Text">common/avro/heterogeneous-access</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/invalid-avro-files">
+        <placeholder name="adapter" value="hdfs" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additional_Properties" value='("input-format" = "avro-input-format")' />
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>External source error. org.apache.avro.InvalidAvroMagicException: Not an Avro data file</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/invalid-type">
+        <placeholder name="adapter" value="hdfs" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additional_Properties" value='("input-format" = "avro-input-format")' />
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX3123: Type 'AvroType' contains declared fields, which is not supported for 'avro' format</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/missing-fields">
+        <placeholder name="adapter" value="hdfs" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additional_Properties" value='("input-format" = "avro-input-format")' />
+        <output-dir compare="Text">common/avro/missing-fields</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/multi-file-multi-schema">
+        <placeholder name="adapter" value="hdfs" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additional_Properties" value='("input-format" = "avro-input-format")' />
+        <output-dir compare="Text">common/avro/multi-file-multi-schema</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/no-files">
+        <placeholder name="adapter" value="hdfs" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additional_Properties" value='("input-format" = "avro-input-format")' />
+        <output-dir compare="Text">common/avro/no-files</output-dir>
+        <expected-error>ASX1114: The provided external dataset configuration returned no files from the external source</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/object-concat">
+        <placeholder name="adapter" value="hdfs" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additional_Properties" value='("input-format" = "avro-input-format")' />
+        <output-dir compare="Text">common/avro/object-concat</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset/common/dynamic-prefixes/avro">
+      <compilation-unit name="one-field">
+        <placeholder name="adapter" value="hdfs" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additional_Properties" value='("input-format" = "avro-input-format")' />
+        <output-dir compare="Text">one-field</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset/common/dynamic-prefixes/avro">
+      <compilation-unit name="query">
+        <placeholder name="adapter" value="hdfs" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additional_Properties" value='("input-format" = "avro-input-format")' />
+        <output-dir compare="Text">query</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset/common/dynamic-prefixes/avro">
+      <compilation-unit name="embed-one-value">
+        <placeholder name="adapter" value="hdfs" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additional_Properties" value='("input-format" = "avro-input-format")' />
+        <output-dir compare="Text">embed-one-value</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset/common/dynamic-prefixes/avro">
+      <compilation-unit name="embed-multiple-values">
+        <placeholder name="adapter" value="hdfs" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additional_Properties" value='("input-format" = "avro-input-format")' />
+        <output-dir compare="Text">embed-multiple-values</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset/common/dynamic-prefixes/avro">
+      <compilation-unit name="embed-flat">
+        <placeholder name="adapter" value="hdfs" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additional_Properties" value='("input-format" = "avro-input-format")' />
+        <output-dir compare="Text">embed-flat</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset/common/dynamic-prefixes/avro">
+      <compilation-unit name="embed-with-nested-values">
+        <placeholder name="adapter" value="hdfs" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additional_Properties" value='("input-format" = "avro-input-format")' />
+        <output-dir compare="Text">embed-with-nested-values</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset/common/dynamic-prefixes/avro">
+      <compilation-unit name="using-limit">
+        <placeholder name="adapter" value="hdfs" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additional_Properties" value='("input-format" = "avro-input-format")' />
+        <output-dir compare="Text">using-limit</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset/common/dynamic-prefixes/avro">
+      <compilation-unit name="views">
+        <placeholder name="adapter" value="hdfs" />
+        <placeholder name="path_prefix" value="/playground/" />
+        <placeholder name="additional_Properties" value='("input-format" = "avro-input-format")' />
+        <output-dir compare="Text">views</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
 </test-suite>
diff --git a/asterixdb/asterix-cloud/pom.xml b/asterixdb/asterix-cloud/pom.xml
index 0ab291d..6f952b0 100644
--- a/asterixdb/asterix-cloud/pom.xml
+++ b/asterixdb/asterix-cloud/pom.xml
@@ -16,181 +16,258 @@
  ! specific language governing permissions and limitations
  ! under the License.
  !-->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <artifactId>apache-asterixdb</artifactId>
-    <groupId>org.apache.asterix</groupId>
-    <version>0.9.10-SNAPSHOT</version>
-  </parent>
-  <artifactId>asterix-cloud</artifactId>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <artifactId>apache-asterixdb</artifactId>
+        <groupId>org.apache.asterix</groupId>
+        <version>0.9.10-SNAPSHOT</version>
+    </parent>
+    <artifactId>asterix-cloud</artifactId>
 
-  <licenses>
-    <license>
-      <name>Apache License, Version 2.0</name>
-      <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
-      <distribution>repo</distribution>
-      <comments>A business-friendly OSS license</comments>
-    </license>
-  </licenses>
+    <licenses>
+        <license>
+            <name>Apache License, Version 2.0</name>
+            <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+            <distribution>repo</distribution>
+            <comments>A business-friendly OSS license</comments>
+        </license>
+    </licenses>
 
-  <properties>
-    <root.dir>${basedir}/..</root.dir>
-  </properties>
+    <properties>
+        <root.dir>${basedir}/..</root.dir>
+    </properties>
 
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.rat</groupId>
-        <artifactId>apache-rat-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>default</id>
-            <phase>validate</phase>
-            <goals>
-              <goal>check</goal>
-            </goals>
-            <configuration>
-              <licenses>
-                <license implementation="org.apache.rat.analysis.license.ApacheSoftwareLicense20"/>
-              </licenses>
-              <excludes combine.children="append">
-                <exclude>src/test/resources/result/**</exclude>
-              </excludes>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>com.googlecode.maven-download-plugin</groupId>
-        <artifactId>download-maven-plugin</artifactId>
-        <version>1.4.2</version>
-        <executions>
-          <execution>
-            <id>install-fake-gcs</id>
-            <phase>${gcs.download.stage}</phase>
-            <goals>
-              <goal>wget</goal>
-            </goals>
-            <configuration>
-              <url>https://github.com/fsouza/fake-gcs-server/releases/download/v1.48.0/fake-gcs-server_1.48.0_Linux_amd64.tar.gz</url>
-              <outputFileName>fake-gcs-server_1.48.0_Linux_amd64.tar.gz</outputFileName>
-              <outputDirectory>${project.build.directory}</outputDirectory>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-antrun-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>extract-gcs</id>
-            <phase>${gcs.install.stage}</phase>
-            <configuration>
-              <target>
-                <echo message="Extracting fake-gcs-server" />
-                <mkdir dir="${project.build.directory}/fake-gcs-server" />
-                <gunzip src="${project.build.directory}/fake-gcs-server_1.48.0_Linux_amd64.tar.gz" dest="${project.build.directory}/fake-gcs-server_1.48.0_Linux_amd64.tar" />
-                <untar src="${project.build.directory}/fake-gcs-server_1.48.0_Linux_amd64.tar" dest="${project.build.directory}/fake-gcs-server" />
-                <chmod file="${project.build.directory}/fake-gcs-server/fake-gcs-server" perm="ugo+rx" />
-              </target>
-            </configuration>
-            <goals>
-              <goal>run</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-      <groupId>org.codehaus.mojo</groupId>
-      <artifactId>exec-maven-plugin</artifactId>
-        <executions>
-          <execution>
-          <id>fake-gcs-server</id>
-          <phase>${gcs.stage}</phase>
-          <goals>
-            <goal>exec</goal>
-          </goals>
-          <configuration>
-            <executable>${project.build.directory}/fake-gcs-server/fake-gcs-server</executable>
-            <workingDirectory>${project.build.directory}/fake-gcs-server</workingDirectory>
-            <arguments>
-              <argument>-port</argument>
-              <argument>24443</argument>
-              <argument>-scheme</argument>
-              <argument>http</argument>
-              <argument>-host</argument>
-              <argument>127.0.0.1</argument>
-              <argument>-log-level</argument>
-              <argument>error</argument>
-              <argument>-filesystem-root</argument>
-              <argument>${project.build.directory}/fake-gcs-server/storage</argument>
-            </arguments>
-            <async>true</async>
-          </configuration>
-          </execution>
-        </executions>
-      </plugin>
-    </plugins>
-  </build>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.rat</groupId>
+                <artifactId>apache-rat-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>default</id>
+                        <phase>validate</phase>
+                        <goals>
+                            <goal>check</goal>
+                        </goals>
+                        <configuration>
+                            <licenses>
+                                <license implementation="org.apache.rat.analysis.license.ApacheSoftwareLicense20"/>
+                            </licenses>
+                            <excludes combine.children="append">
+                                <exclude>src/test/resources/result/**</exclude>
+                            </excludes>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>com.googlecode.maven-download-plugin</groupId>
+                <artifactId>download-maven-plugin</artifactId>
+                <version>1.4.2</version>
+                <executions>
+                    <execution>
+                        <id>install-fake-gcs</id>
+                        <phase>${gcs.download.stage}</phase>
+                        <goals>
+                            <goal>wget</goal>
+                        </goals>
+                        <configuration>
+                            <url>
+                                https://github.com/fsouza/fake-gcs-server/releases/download/v1.48.0/fake-gcs-server_1.48.0_Linux_amd64.tar.gz
+                            </url>
+                            <outputFileName>fake-gcs-server_1.48.0_Linux_amd64.tar.gz</outputFileName>
+                            <outputDirectory>${project.build.directory}</outputDirectory>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-antrun-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>extract-gcs</id>
+                        <phase>${gcs.install.stage}</phase>
+                        <configuration>
+                            <target>
+                                <echo message="Extracting fake-gcs-server"/>
+                                <mkdir dir="${project.build.directory}/fake-gcs-server"/>
+                                <gunzip src="${project.build.directory}/fake-gcs-server_1.48.0_Linux_amd64.tar.gz"
+                                        dest="${project.build.directory}/fake-gcs-server_1.48.0_Linux_amd64.tar"/>
+                                <untar src="${project.build.directory}/fake-gcs-server_1.48.0_Linux_amd64.tar"
+                                       dest="${project.build.directory}/fake-gcs-server"/>
+                                <chmod file="${project.build.directory}/fake-gcs-server/fake-gcs-server" perm="ugo+rx"/>
+                            </target>
+                        </configuration>
+                        <goals>
+                            <goal>run</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>exec-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>fake-gcs-server</id>
+                        <phase>${gcs.stage}</phase>
+                        <goals>
+                            <goal>exec</goal>
+                        </goals>
+                        <configuration>
+                            <executable>${project.build.directory}/fake-gcs-server/fake-gcs-server</executable>
+                            <workingDirectory>${project.build.directory}/fake-gcs-server</workingDirectory>
+                            <arguments>
+                                <argument>-port</argument>
+                                <argument>24443</argument>
+                                <argument>-scheme</argument>
+                                <argument>http</argument>
+                                <argument>-host</argument>
+                                <argument>127.0.0.1</argument>
+                                <argument>-log-level</argument>
+                                <argument>error</argument>
+                                <argument>-filesystem-root</argument>
+                                <argument>${project.build.directory}/fake-gcs-server/storage</argument>
+                            </arguments>
+                            <async>true</async>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>azurite</id>
+                        <phase>${azurite.stage}</phase>
+                        <goals>
+                            <goal>exec</goal>
+                        </goals>
+                        <configuration>
+                            <!--suppress UnresolvedMavenProperty -->
+                            <executable>${project.build.directory}/npm/node_modules/.bin/azurite-blob</executable>
+                            <workingDirectory>${project.build.directory}</workingDirectory>
+                            <environmentVariables>
+                                <PATH>${project.build.directory}/npm/node</PATH>
+                            </environmentVariables>
+                            <arguments>
+                                <argument>--blobPort</argument>
+                                <argument>15055</argument>
+                                <argument>--location</argument>
+                                <argument>${project.build.directory}/azurite</argument>
+                                <argument>--debug</argument>
+                                <argument>${project.build.directory}/azurite/logs/azurite-debug.log</argument>
+                            </arguments>
+                            <async>true</async>
+                            <outputFile>${project.build.directory}/azurite/logs/azurite.log</outputFile>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
 
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.hyracks</groupId>
-      <artifactId>hyracks-cloud</artifactId>
-      <version>${hyracks.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.asterix</groupId>
-      <artifactId>asterix-common</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.asterix</groupId>
-      <artifactId>asterix-external-data</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <!-- aws s3 start -->
-    <dependency>
-      <groupId>software.amazon.awssdk</groupId>
-      <artifactId>sdk-core</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>software.amazon.awssdk</groupId>
-      <artifactId>s3</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>software.amazon.awssdk</groupId>
-      <artifactId>regions</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>software.amazon.awssdk</groupId>
-      <artifactId>auth</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>software.amazon.awssdk</groupId>
-      <artifactId>s3-transfer-manager</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>software.amazon.awssdk.crt</groupId>
-      <artifactId>aws-crt</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>io.findify</groupId>
-      <artifactId>s3mock_2.12</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>com.typesafe.akka</groupId>
-      <artifactId>akka-http-core_2.12</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <!-- aws s3 end -->
-  </dependencies>
+    <profiles>
+        <profile>
+            <id>azurite-tests</id>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>com.github.eirslett</groupId>
+                        <artifactId>frontend-maven-plugin</artifactId>
+                        <version>1.13.4</version>
+                        <configuration>
+                            <nodeVersion>v14.15.4</nodeVersion>
+                            <npmVersion>6.14.11</npmVersion>
+                            <workingDirectory>target/npm</workingDirectory>
+                            <installDirectory>target/npm</installDirectory>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <id>install node and yarn</id>
+                                <goals>
+                                    <goal>install-node-and-npm</goal>
+                                </goals>
+                                <phase>${azurite.npm.install.stage}</phase>
+                            </execution>
+                            <execution>
+                                <id>azurite blob</id>
+                                <phase>${azurite.install.stage}</phase>
+                                <goals>
+                                    <goal>npm</goal>
+                                </goals>
+                                <configuration>
+                                    <arguments>install azurite@3.31.0</arguments>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+    </profiles>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.hyracks</groupId>
+            <artifactId>hyracks-cloud</artifactId>
+            <version>${hyracks.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.asterix</groupId>
+            <artifactId>asterix-common</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.asterix</groupId>
+            <artifactId>asterix-external-data</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <!-- aws s3 start -->
+        <dependency>
+            <groupId>software.amazon.awssdk</groupId>
+            <artifactId>sdk-core</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>software.amazon.awssdk</groupId>
+            <artifactId>s3</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>software.amazon.awssdk</groupId>
+            <artifactId>regions</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>software.amazon.awssdk</groupId>
+            <artifactId>auth</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>software.amazon.awssdk</groupId>
+            <artifactId>s3-transfer-manager</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>software.amazon.awssdk.crt</groupId>
+            <artifactId>aws-crt</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>io.findify</groupId>
+            <artifactId>s3mock_2.12</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.typesafe.akka</groupId>
+            <artifactId>akka-http-core_2.12</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <!-- aws s3 end -->
+
+        <dependency>
+            <groupId>com.azure</groupId>
+            <artifactId>azure-storage-blob-batch</artifactId>
+            <version>12.23.0</version>
+        </dependency>
+
+    </dependencies>
 </project>
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/AbstractCloudIOManager.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/AbstractCloudIOManager.java
index c005253..4ce382e 100644
--- a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/AbstractCloudIOManager.java
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/AbstractCloudIOManager.java
@@ -26,10 +26,12 @@
 import java.io.IOException;
 import java.io.InputStream;
 import java.nio.ByteBuffer;
+import java.nio.file.FileStore;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
+import java.util.function.Predicate;
 
 import org.apache.asterix.cloud.bulk.DeleteBulkCloudOperation;
 import org.apache.asterix.cloud.bulk.NoOpDeleteBulkCallBack;
@@ -51,6 +53,7 @@
 import org.apache.hyracks.api.io.IIOBulkOperation;
 import org.apache.hyracks.api.io.IODeviceHandle;
 import org.apache.hyracks.api.util.IoUtil;
+import org.apache.hyracks.cloud.filesystem.PhysicalDrive;
 import org.apache.hyracks.cloud.io.ICloudIOManager;
 import org.apache.hyracks.cloud.io.request.ICloudBeforeRetryRequest;
 import org.apache.hyracks.cloud.io.request.ICloudRequest;
@@ -68,6 +71,8 @@
 public abstract class AbstractCloudIOManager extends IOManager implements IPartitionBootstrapper, ICloudIOManager {
     private static final Logger LOGGER = LogManager.getLogger();
     private static final byte[] EMPTY_FILE_BYTES = "empty".getBytes();
+    private static final Predicate<String> NO_OP_LIST_FILES_FILTER = (path) -> true;
+
     protected final ICloudClient cloudClient;
     protected final ICloudGuardian guardian;
     protected final IWriteBufferProvider writeBufferProvider;
@@ -76,6 +81,7 @@
     protected final List<FileReference> partitionPaths;
     protected final IOManager localIoManager;
     protected final INamespacePathResolver nsPathResolver;
+    private final List<FileStore> drivePaths;
 
     public AbstractCloudIOManager(IOManager ioManager, CloudProperties cloudProperties,
             INamespacePathResolver nsPathResolver, ICloudGuardian guardian) throws HyracksDataException {
@@ -90,6 +96,7 @@
         partitions = new HashSet<>();
         partitionPaths = new ArrayList<>();
         this.localIoManager = ioManager;
+        drivePaths = PhysicalDrive.getDrivePaths(ioDevices);
     }
 
     /*
@@ -468,10 +475,24 @@
     }
 
     public long getTotalRemoteStorageSizeForNodeBytes() {
-        long size = 0;
-        for (CloudFile file : list()) {
-            size += file.getSize();
+        return getSize(NO_OP_LIST_FILES_FILTER);
+    }
+
+    @Override
+    public long getSize(Predicate<String> relativePathFilter) {
+        long totalSize = localIoManager.getSize(relativePathFilter);
+
+        // get uncached files from uncached files tracker
+        for (UncachedFileReference uncachedFile : getUncachedFiles()) {
+            if (relativePathFilter.test(uncachedFile.getRelativePath())) {
+                totalSize += uncachedFile.getSize();
+            }
         }
-        return size;
+        return totalSize;
+    }
+
+    @Override
+    public long getTotalDiskUsage() {
+        return PhysicalDrive.getUsedSpace(drivePaths);
     }
 }
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/AbstractCloudRequestRateLimiter.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/AbstractCloudRequestRateLimiter.java
new file mode 100644
index 0000000..b1c1c94
--- /dev/null
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/AbstractCloudRequestRateLimiter.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.cloud.clients;
+
+import org.apache.asterix.cloud.clients.profiler.limiter.IRateLimiter;
+import org.apache.asterix.cloud.clients.profiler.limiter.IRequestRateLimiter;
+import org.apache.asterix.cloud.clients.profiler.limiter.NoOpRateLimiter;
+import org.apache.asterix.cloud.clients.profiler.limiter.TokenBasedRateLimiter;
+
+public abstract class AbstractCloudRequestRateLimiter implements IRequestRateLimiter {
+
+    protected final IRateLimiter writeLimiter;
+    protected final IRateLimiter readLimiter;
+
+    public AbstractCloudRequestRateLimiter(int writeMaxRequestsPerSeconds, int readMaxRequestsPerSeconds,
+            long tokenAcquireTimeout) {
+        this.writeLimiter = createLimiter(writeMaxRequestsPerSeconds, tokenAcquireTimeout);
+        this.readLimiter = createLimiter(readMaxRequestsPerSeconds, tokenAcquireTimeout);
+    }
+
+    @Override
+    public void writeRequest() {
+        writeLimiter.acquire();
+    }
+
+    @Override
+    public void readRequest() {
+        readLimiter.acquire();
+    }
+
+    @Override
+    public void listRequest() {
+        readLimiter.acquire();
+    }
+
+    @Override
+    public long getReadThrottleCount() {
+        return readLimiter.getThrottleCount();
+    }
+
+    @Override
+    public long getWriteThrottleCount() {
+        return writeLimiter.getThrottleCount();
+    }
+
+    private static IRateLimiter createLimiter(int maxRequestsPerSecond, long tokeAcquireTimeout) {
+        if (maxRequestsPerSecond > 0) {
+            return new TokenBasedRateLimiter(maxRequestsPerSecond, tokeAcquireTimeout);
+        }
+        return NoOpRateLimiter.INSTANCE;
+    }
+}
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/CloudClientProvider.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/CloudClientProvider.java
index ee43a2c..c98c6b4 100644
--- a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/CloudClientProvider.java
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/CloudClientProvider.java
@@ -20,6 +20,8 @@
 
 import org.apache.asterix.cloud.clients.aws.s3.S3ClientConfig;
 import org.apache.asterix.cloud.clients.aws.s3.S3CloudClient;
+import org.apache.asterix.cloud.clients.azure.blobstorage.AzBlobStorageClientConfig;
+import org.apache.asterix.cloud.clients.azure.blobstorage.AzBlobStorageCloudClient;
 import org.apache.asterix.cloud.clients.google.gcs.GCSClientConfig;
 import org.apache.asterix.cloud.clients.google.gcs.GCSCloudClient;
 import org.apache.asterix.common.config.CloudProperties;
@@ -30,6 +32,7 @@
     private static final boolean UNSTABLE = isUnstable();
     public static final String S3 = "s3";
     public static final String GCS = "gs";
+    public static final String AZ_BLOB = "azblob";
 
     private CloudClientProvider() {
         throw new AssertionError("do not instantiate");
@@ -45,6 +48,9 @@
         } else if (GCS.equalsIgnoreCase(storageScheme)) {
             GCSClientConfig config = GCSClientConfig.of(cloudProperties);
             cloudClient = new GCSCloudClient(config, guardian);
+        } else if (AZ_BLOB.equalsIgnoreCase(storageScheme)) {
+            AzBlobStorageClientConfig config = AzBlobStorageClientConfig.of(cloudProperties);
+            cloudClient = new AzBlobStorageCloudClient(config, guardian);
         } else {
             throw new IllegalStateException("unsupported cloud storage scheme: " + storageScheme);
         }
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/aws/s3/S3RequestRateLimiter.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/aws/s3/S3RequestRateLimiter.java
index 37387a6..c2d730b 100644
--- a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/aws/s3/S3RequestRateLimiter.java
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/aws/s3/S3RequestRateLimiter.java
@@ -18,41 +18,12 @@
  */
 package org.apache.asterix.cloud.clients.aws.s3;
 
-import org.apache.asterix.cloud.clients.profiler.limiter.IRateLimiter;
-import org.apache.asterix.cloud.clients.profiler.limiter.IRequestRateLimiter;
-import org.apache.asterix.cloud.clients.profiler.limiter.NoOpRateLimiter;
-import org.apache.asterix.cloud.clients.profiler.limiter.TokenBasedRateLimiter;
+import org.apache.asterix.cloud.clients.AbstractCloudRequestRateLimiter;
 
-public final class S3RequestRateLimiter implements IRequestRateLimiter {
-    private final IRateLimiter writeLimiter;
-    private final IRateLimiter readLimiter;
+public final class S3RequestRateLimiter extends AbstractCloudRequestRateLimiter {
 
     public S3RequestRateLimiter(S3ClientConfig config) {
-        long tokenAcquireTimeout = config.getTokenAcquireTimeout();
-        this.writeLimiter = createLimiter(config.getWriteMaxRequestsPerSeconds(), tokenAcquireTimeout);
-        this.readLimiter = createLimiter(config.getReadMaxRequestsPerSeconds(), tokenAcquireTimeout);
-    }
-
-    @Override
-    public void writeRequest() {
-        writeLimiter.acquire();
-    }
-
-    @Override
-    public void readRequest() {
-        readLimiter.acquire();
-    }
-
-    @Override
-    public void listRequest() {
-        // List requests in S3 are considered as PUT
-        writeLimiter.acquire();
-    }
-
-    private static IRateLimiter createLimiter(int maxRequestsPerSecond, long tokeAcquireTimeout) {
-        if (maxRequestsPerSecond > 0) {
-            return new TokenBasedRateLimiter(maxRequestsPerSecond, tokeAcquireTimeout);
-        }
-        return NoOpRateLimiter.INSTANCE;
+        super(config.getWriteMaxRequestsPerSeconds(), config.getReadMaxRequestsPerSeconds(),
+                config.getTokenAcquireTimeout());
     }
 }
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzBlobStorageBufferedWriter.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzBlobStorageBufferedWriter.java
new file mode 100644
index 0000000..2a79c86
--- /dev/null
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzBlobStorageBufferedWriter.java
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.cloud.clients.azure.blobstorage;
+
+import java.io.BufferedInputStream;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Base64;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.asterix.cloud.clients.ICloudBufferedWriter;
+import org.apache.asterix.cloud.clients.ICloudGuardian;
+import org.apache.asterix.cloud.clients.profiler.IRequestProfilerLimiter;
+import org.apache.commons.io.IOUtils;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import com.azure.core.util.BinaryData;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.models.BlobStorageException;
+import com.azure.storage.blob.specialized.BlockBlobClient;
+
+public class AzBlobStorageBufferedWriter implements ICloudBufferedWriter {
+    private static final String PUT_UPLOAD_ID = "putUploadId";
+    private static final int MAX_RETRIES = 3;
+    private static final Logger LOGGER = LogManager.getLogger();
+    private final List<String> blockIDArrayList;
+    private final ICloudGuardian guardian;
+    private int blockNumber;
+    private final String path;
+    private String uploadID;
+
+    private final BlobContainerClient blobContainerClient;
+
+    private final IRequestProfilerLimiter profiler;
+
+    private final String bucket;
+
+    public AzBlobStorageBufferedWriter(BlobContainerClient blobContainerClient, IRequestProfilerLimiter profiler,
+            ICloudGuardian guardian, String bucket, String path) {
+        this.blobContainerClient = blobContainerClient;
+        this.profiler = profiler;
+        this.guardian = guardian;
+        this.bucket = bucket;
+        this.path = path;
+        this.blockIDArrayList = new ArrayList<>();
+    }
+
+    @Override
+    public void upload(InputStream stream, int length) {
+        profiler.objectMultipartUpload();
+        if (length <= 0) {
+            String errMsg = String.format("A block with size %d cannot be staged for upload", length);
+            LOGGER.error(errMsg);
+            throw new IllegalArgumentException(errMsg);
+        }
+        guardian.checkIsolatedWriteAccess(bucket, path);
+        try {
+            BlockBlobClient blockBlobClient = blobContainerClient.getBlobClient(path).getBlockBlobClient();
+            BufferedInputStream bufferedInputStream = IOUtils.buffer(stream, length);
+            String blockID =
+                    Base64.getEncoder().encodeToString(UUID.randomUUID().toString().getBytes(StandardCharsets.UTF_8));
+            initBlockBlobUploads(blockID);
+            blockIDArrayList.add(blockID);
+            blockBlobClient.stageBlock(blockID, bufferedInputStream, length);
+        } catch (Exception e) {
+            LOGGER.error("Error while uploading blocks of data: {}", e.getMessage());
+            throw new RuntimeException(e);
+        }
+        blockNumber++;
+    }
+
+    private void initBlockBlobUploads(String blockID) {
+        if (this.uploadID == null) {
+            this.uploadID = blockID;
+            this.blockNumber = 1;
+        }
+    }
+
+    @Override
+    public void uploadLast(InputStream stream, ByteBuffer buffer) throws HyracksDataException {
+        if (uploadID == null) {
+            profiler.objectWrite();
+            BlobClient blobClient = blobContainerClient.getBlobClient(path);
+            BinaryData binaryData = BinaryData.fromBytes(getDataFromBuffer(buffer));
+            blobClient.upload(binaryData);
+            uploadID = PUT_UPLOAD_ID; // uploadID should be updated if the put-object operation succeeds
+        } else {
+            upload(stream, buffer.limit());
+        }
+    }
+
+    private byte[] getDataFromBuffer(ByteBuffer buffer) {
+        byte[] data = new byte[buffer.limit()];
+        buffer.get(data, 0, buffer.limit());
+        return data;
+    }
+
+    @Override
+    public boolean isEmpty() {
+        return this.uploadID == null;
+    }
+
+    @Override
+    public void finish() throws HyracksDataException {
+        if (this.uploadID == null) {
+            throw new IllegalStateException("Cannot finish without writing any bytes");
+        } else if (PUT_UPLOAD_ID.equals(uploadID)) {
+            return;
+        }
+        int currRetryAttempt = 0;
+        BlockBlobClient blockBlobClient = blobContainerClient.getBlobClient(path).getBlockBlobClient();
+        while (true) {
+            try {
+                guardian.checkWriteAccess(bucket, path);
+                profiler.objectMultipartUpload();
+                blockBlobClient.commitBlockList(blockIDArrayList);
+                break;
+            } catch (BlobStorageException e) {
+                currRetryAttempt++;
+                if (currRetryAttempt == MAX_RETRIES) {
+                    throw HyracksDataException.create(e);
+                }
+                LOGGER.info(() -> "AzBlob storage write retry, encountered: " + e.getMessage());
+
+                // Backoff for 1 sec for the first 2 retries, and 2 seconds from there onward
+                try {
+                    Thread.sleep(TimeUnit.SECONDS.toMillis(currRetryAttempt < 2 ? 1 : 2));
+                } catch (InterruptedException ex) {
+                    Thread.currentThread().interrupt();
+                    throw HyracksDataException.create(ex);
+                }
+            }
+        }
+    }
+
+    @Override
+    public void abort() throws HyracksDataException {
+        // Todo: As of the current Azure Java SDK, it does not support aborting a staged or under-upload block.
+        // https://github.com/Azure/azure-sdk-for-java/issues/31150
+        LOGGER.warn("Multipart upload for {} was aborted", path);
+    }
+}
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzBlobStorageClientConfig.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzBlobStorageClientConfig.java
new file mode 100644
index 0000000..9aedfc3
--- /dev/null
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzBlobStorageClientConfig.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.cloud.clients.azure.blobstorage;
+
+import java.util.Objects;
+
+import org.apache.asterix.common.config.CloudProperties;
+
+import com.azure.identity.DefaultAzureCredential;
+import com.azure.identity.DefaultAzureCredentialBuilder;
+
+public class AzBlobStorageClientConfig {
+    private final int writeBufferSize;
+    // Ref: https://learn.microsoft.com/en-us/rest/api/storageservices/blob-batch?tabs=microsoft-entra-id
+    static final int DELETE_BATCH_SIZE = 256;
+    private final String region;
+    private final String endpoint;
+    private final String prefix;
+
+    private final boolean anonymousAuth;
+    private final long profilerLogInterval;
+    private final String bucket;
+    private final long tokenAcquireTimeout;
+    private final int writeMaxRequestsPerSeconds;
+    private final int readMaxRequestsPerSeconds;
+
+    public AzBlobStorageClientConfig(String region, String endpoint, String prefix, boolean anonymousAuth,
+            long profilerLogInterval, String bucket, long tokenAcquireTimeout, int writeMaxRequestsPerSeconds,
+            int readMaxRequestsPerSeconds, int writeBufferSize) {
+        this.region = Objects.requireNonNull(region, "region");
+        this.endpoint = endpoint;
+        this.prefix = Objects.requireNonNull(prefix, "prefix");
+        this.anonymousAuth = anonymousAuth;
+        this.profilerLogInterval = profilerLogInterval;
+        this.bucket = bucket;
+        this.tokenAcquireTimeout = tokenAcquireTimeout;
+        this.writeMaxRequestsPerSeconds = writeMaxRequestsPerSeconds;
+        this.readMaxRequestsPerSeconds = readMaxRequestsPerSeconds;
+        this.writeBufferSize = writeBufferSize;
+    }
+
+    public static AzBlobStorageClientConfig of(CloudProperties cloudProperties) {
+        return new AzBlobStorageClientConfig(cloudProperties.getStorageRegion(), cloudProperties.getStorageEndpoint(),
+                cloudProperties.getStoragePrefix(), cloudProperties.isStorageAnonymousAuth(),
+                cloudProperties.getProfilerLogInterval(), cloudProperties.getStorageBucket(),
+                cloudProperties.getTokenAcquireTimeout(), cloudProperties.getWriteMaxRequestsPerSecond(),
+                cloudProperties.getReadMaxRequestsPerSecond(), cloudProperties.getWriteBufferSize());
+    }
+
+    public String getRegion() {
+        return region;
+    }
+
+    public String getEndpoint() {
+        return endpoint;
+    }
+
+    public String getPrefix() {
+        return prefix;
+    }
+
+    public String getBucket() {
+        return bucket;
+    }
+
+    public long getProfilerLogInterval() {
+        return profilerLogInterval;
+    }
+
+    public boolean isAnonymousAuth() {
+        return anonymousAuth;
+    }
+
+    public DefaultAzureCredential createCredentialsProvider() {
+        return new DefaultAzureCredentialBuilder().build();
+    }
+
+    public long getTokenAcquireTimeout() {
+        return tokenAcquireTimeout;
+    }
+
+    public int getWriteMaxRequestsPerSeconds() {
+        return writeMaxRequestsPerSeconds;
+    }
+
+    public int getReadMaxRequestsPerSeconds() {
+        return readMaxRequestsPerSeconds;
+    }
+
+    public int getWriteBufferSize() {
+        return writeBufferSize;
+    }
+}
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzBlobStorageCloudClient.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzBlobStorageCloudClient.java
new file mode 100644
index 0000000..b9f9421
--- /dev/null
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzBlobStorageCloudClient.java
@@ -0,0 +1,402 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.cloud.clients.azure.blobstorage;
+
+import static org.apache.asterix.cloud.clients.azure.blobstorage.AzBlobStorageClientConfig.DELETE_BATCH_SIZE;
+
+import java.io.ByteArrayOutputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.BufferOverflowException;
+import java.nio.ByteBuffer;
+import java.nio.ReadOnlyBufferException;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.apache.asterix.cloud.CloudResettableInputStream;
+import org.apache.asterix.cloud.IWriteBufferProvider;
+import org.apache.asterix.cloud.clients.CloudFile;
+import org.apache.asterix.cloud.clients.ICloudBufferedWriter;
+import org.apache.asterix.cloud.clients.ICloudClient;
+import org.apache.asterix.cloud.clients.ICloudGuardian;
+import org.apache.asterix.cloud.clients.ICloudWriter;
+import org.apache.asterix.cloud.clients.IParallelDownloader;
+import org.apache.asterix.cloud.clients.profiler.CountRequestProfilerLimiter;
+import org.apache.asterix.cloud.clients.profiler.IRequestProfilerLimiter;
+import org.apache.asterix.cloud.clients.profiler.RequestLimiterNoOpProfiler;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.control.nc.io.IOManager;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import com.azure.core.http.rest.PagedIterable;
+import com.azure.core.util.BinaryData;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.BlobContainerClientBuilder;
+import com.azure.storage.blob.batch.BlobBatchClient;
+import com.azure.storage.blob.batch.BlobBatchClientBuilder;
+import com.azure.storage.blob.models.BlobErrorCode;
+import com.azure.storage.blob.models.BlobItem;
+import com.azure.storage.blob.models.BlobListDetails;
+import com.azure.storage.blob.models.BlobRange;
+import com.azure.storage.blob.models.BlobStorageException;
+import com.azure.storage.blob.models.ListBlobsOptions;
+import com.azure.storage.common.StorageSharedKeyCredential;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+public class AzBlobStorageCloudClient implements ICloudClient {
+    private static final String BUCKET_ROOT_PATH = "";
+    public static final String AZURITE_ENDPOINT = "http://127.0.0.1:15055/devstoreaccount1/";
+    private static final String AZURITE_ACCOUNT_NAME = "devstoreaccount1";
+    private static final String AZURITE_ACCOUNT_KEY =
+            "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==";
+    private final ICloudGuardian guardian;
+    private BlobContainerClient blobContainerClient;
+    private AzBlobStorageClientConfig config;
+    private IRequestProfilerLimiter profiler;
+    private final BlobBatchClient blobBatchClient;
+    private static final Logger LOGGER = LogManager.getLogger();
+
+    public AzBlobStorageCloudClient(AzBlobStorageClientConfig config, ICloudGuardian guardian) {
+        this(config, buildClient(config), guardian);
+    }
+
+    public AzBlobStorageCloudClient(AzBlobStorageClientConfig config, BlobContainerClient blobContainerClient,
+            ICloudGuardian guardian) {
+        this.blobContainerClient = blobContainerClient;
+        this.config = config;
+        this.guardian = guardian;
+        long profilerInterval = config.getProfilerLogInterval();
+        AzureRequestRateLimiter limiter = new AzureRequestRateLimiter(config);
+        if (profilerInterval > 0) {
+            profiler = new CountRequestProfilerLimiter(profilerInterval, limiter);
+        } else {
+            profiler = new RequestLimiterNoOpProfiler(limiter);
+        }
+        guardian.setCloudClient(this);
+        blobBatchClient = new BlobBatchClientBuilder(blobContainerClient.getServiceClient()).buildClient();
+    }
+
+    @Override
+    public int getWriteBufferSize() {
+        return config.getWriteBufferSize();
+    }
+
+    @Override
+    public IRequestProfilerLimiter getProfilerLimiter() {
+        return profiler;
+    }
+
+    @Override
+    public ICloudWriter createWriter(String bucket, String path, IWriteBufferProvider bufferProvider) {
+        ICloudBufferedWriter bufferedWriter = new AzBlobStorageBufferedWriter(blobContainerClient, profiler, guardian,
+                bucket, config.getPrefix() + path);
+        return new CloudResettableInputStream(bufferedWriter, bufferProvider);
+    }
+
+    @Override
+    public Set<CloudFile> listObjects(String bucket, String path, FilenameFilter filter) {
+        guardian.checkReadAccess(bucket, path);
+        profiler.objectsList();
+        PagedIterable<BlobItem> blobItems = getBlobItems(bucket, config.getPrefix() + path);
+        Stream<CloudFile> cloudFileStream = mapBlobItemsToStreamOfCloudFiles(blobItems);
+        return filterCloudFiles(filter, cloudFileStream);
+    }
+
+    private Set<CloudFile> filterCloudFiles(FilenameFilter filter, Stream<CloudFile> cloudFileStream) {
+        if (filter == null) {
+            return cloudFileStream.map(this::removeCloudPrefixFromBlobName).collect(Collectors.toSet());
+        }
+        return cloudFileStream.filter(cloudFile -> filter.accept(null, cloudFile.getPath()))
+                .map(this::removeCloudPrefixFromBlobName).collect(Collectors.toSet());
+    }
+
+    private CloudFile removeCloudPrefixFromBlobName(CloudFile cloudFile) {
+        String fullyQualifiedBlobName = cloudFile.getPath();
+        fullyQualifiedBlobName = fullyQualifiedBlobName.substring(config.getPrefix().length());
+        return CloudFile.of(fullyQualifiedBlobName, cloudFile.getSize());
+    }
+
+    private Stream<CloudFile> mapBlobItemsToStreamOfCloudFiles(PagedIterable<BlobItem> blobItems) {
+        return blobItems.stream()
+                .map(blobItem -> CloudFile.of(blobItem.getName(), blobItem.getProperties().getContentLength()));
+    }
+
+    private PagedIterable<BlobItem> getBlobItems(String bucket, String path) {
+        ListBlobsOptions options =
+                new ListBlobsOptions().setPrefix(path).setDetails(new BlobListDetails().setRetrieveMetadata(true));
+        return blobContainerClient.listBlobs(options, null);
+    }
+
+    @Override
+    public int read(String bucket, String path, long offset, ByteBuffer buffer) throws HyracksDataException {
+        guardian.checkReadAccess(bucket, path);
+        profiler.objectGet();
+        BlobClient blobClient = blobContainerClient.getBlobClient(config.getPrefix() + path);
+        ByteArrayOutputStream blobStream = new ByteArrayOutputStream(buffer.capacity());
+        long rem = buffer.remaining();
+        BlobRange blobRange = new BlobRange(offset, rem);
+        downloadBlob(blobClient, blobStream, blobRange);
+        readBlobStreamIntoBuffer(buffer, blobStream);
+        if (buffer.remaining() != 0)
+            throw new IllegalStateException("Expected buffer remaining = 0, found: " + buffer.remaining());
+        return ((int) rem - buffer.remaining());
+    }
+
+    private void readBlobStreamIntoBuffer(ByteBuffer buffer, ByteArrayOutputStream byteArrayOutputStream)
+            throws HyracksDataException {
+        byte[] byteArray = byteArrayOutputStream.toByteArray();
+        try {
+            buffer.put(byteArray);
+            byteArrayOutputStream.close();
+        } catch (BufferOverflowException | ReadOnlyBufferException | IOException ex) {
+            throw HyracksDataException.create(ex);
+        }
+    }
+
+    private void downloadBlob(BlobClient blobClient, ByteArrayOutputStream byteArrayOutputStream, BlobRange blobRange)
+            throws HyracksDataException {
+        try {
+            blobClient.downloadStreamWithResponse(byteArrayOutputStream, blobRange, null, null, false, null, null);
+        } catch (BlobStorageException ex) {
+            throw HyracksDataException.create(ex);
+        }
+    }
+
+    @Override
+    public byte[] readAllBytes(String bucket, String path) throws HyracksDataException {
+        guardian.checkReadAccess(bucket, path);
+        profiler.objectGet();
+        BlobClient blobClient = blobContainerClient.getBlobClient(config.getPrefix() + path);
+        try {
+            BinaryData binaryData = blobClient.downloadContent();
+            return binaryData.toBytes();
+        } catch (BlobStorageException ex) {
+            BlobErrorCode errorCode = ex.getErrorCode();
+            if (errorCode.equals(BlobErrorCode.BLOB_NOT_FOUND)) {
+                LOGGER.warn("Blob not found on cloud: {}", path);
+                return null;
+            }
+            throw HyracksDataException.create(ex);
+        }
+    }
+
+    @Override
+    public InputStream getObjectStream(String bucket, String path, long offset, long length) {
+        guardian.checkReadAccess(bucket, path);
+        profiler.objectGet();
+        BlobRange blobRange = new BlobRange(offset, length);
+        BlobClient blobClient = blobContainerClient.getBlobClient(config.getPrefix() + path);
+        try {
+            return blobClient.openInputStream(blobRange, null);
+        } catch (BlobStorageException ex) {
+            LOGGER.error("error getting object stream for path: {}. Exception: {}", path, ex.getMessage());
+            throw new IllegalStateException(ex);
+        }
+    }
+
+    @Override
+    public void write(String bucket, String path, byte[] data) {
+        guardian.checkWriteAccess(bucket, path);
+        profiler.objectWrite();
+        BinaryData binaryData = BinaryData.fromBytes(data);
+        BlobClient blobClient = blobContainerClient.getBlobClient(config.getPrefix() + path);
+        blobClient.upload(binaryData, true);
+    }
+
+    @Override
+    public void copy(String bucket, String srcPath, FileReference destPath) {
+        guardian.checkReadAccess(bucket, srcPath);
+        profiler.objectGet();
+        BlobClient srcBlobClient = blobContainerClient.getBlobClient(config.getPrefix() + srcPath);
+        String srcBlobUrl = srcBlobClient.getBlobUrl();
+        profiler.objectCopy();
+        guardian.checkWriteAccess(bucket, destPath.getRelativePath());
+        BlobClient destBlobClient = blobContainerClient.getBlobClient(destPath.getFile().getPath());
+        destBlobClient.beginCopy(srcBlobUrl, null);
+    }
+
+    @Override
+    public void deleteObjects(String bucket, Collection<String> paths) {
+        if (paths.isEmpty())
+            return;
+        Set<BlobItem> blobsToDelete = getBlobsMatchingThesePaths(paths);
+        List<String> blobURLs = getBlobURLs(blobsToDelete);
+        if (blobURLs.isEmpty())
+            return;
+        Collection<List<String>> batchedBlobURLs = getBatchedBlobURLs(blobURLs);
+        for (List<String> batch : batchedBlobURLs) {
+            blobBatchClient.deleteBlobs(batch, null).stream().count();
+        }
+    }
+
+    private Collection<List<String>> getBatchedBlobURLs(List<String> blobURLs) {
+        int startIdx = 0;
+        Collection<List<String>> batchedBLOBURLs = new ArrayList<>();
+        Iterator<String> iterator = blobURLs.iterator();
+        while (iterator.hasNext()) {
+            List<String> batch = new ArrayList<>();
+            while (startIdx < DELETE_BATCH_SIZE && iterator.hasNext()) {
+                batch.add(iterator.next());
+                startIdx++;
+            }
+            batchedBLOBURLs.add(batch);
+            startIdx = 0;
+        }
+        return batchedBLOBURLs;
+    }
+
+    private Set<BlobItem> getBlobsMatchingThesePaths(Collection<String> paths) {
+        List<String> pathWithPrefix =
+                paths.stream().map(path -> config.getPrefix() + path).collect(Collectors.toList());
+        PagedIterable<BlobItem> blobItems = blobContainerClient.listBlobs();
+        return blobItems.stream().filter(blobItem -> pathWithPrefix.contains(blobItem.getName()))
+                .collect(Collectors.toSet());
+    }
+
+    @Override
+    public long getObjectSize(String bucket, String path) throws HyracksDataException {
+        guardian.checkReadAccess(bucket, path);
+        profiler.objectGet();
+        try {
+            BlobClient blobClient = blobContainerClient.getBlobClient(config.getPrefix() + path);
+            return blobClient.getProperties().getBlobSize();
+        } catch (BlobStorageException ex) {
+            BlobErrorCode errorCode = ex.getErrorCode();
+            if (errorCode.equals(BlobErrorCode.BLOB_NOT_FOUND)) {
+                LOGGER.error("error while getting blob size; no such blob found: {} ", config.getPrefix() + path);
+                return 0;
+            }
+            throw HyracksDataException.create(ex);
+        } catch (Exception ex) {
+            LOGGER.error("error getting size of the blob: {}. Exception: {}", path, ex.getMessage());
+            throw HyracksDataException.create(ex);
+        }
+    }
+
+    @Override
+    public boolean exists(String bucket, String path) throws HyracksDataException {
+        guardian.checkReadAccess(bucket, path);
+        profiler.objectGet();
+        try {
+            BlobClient blobClient = blobContainerClient.getBlobClient(config.getPrefix() + path);
+            return blobClient.exists();
+        } catch (BlobStorageException ex) {
+            BlobErrorCode errorCode = ex.getErrorCode();
+            if (errorCode.equals(BlobErrorCode.BLOB_NOT_FOUND)) {
+                return false;
+            }
+            throw HyracksDataException.create(ex);
+        } catch (Exception ex) {
+            throw HyracksDataException.create(ex);
+        }
+    }
+
+    @Override
+    public boolean isEmptyPrefix(String bucket, String path) throws HyracksDataException {
+        profiler.objectsList();
+        ListBlobsOptions listBlobsOptions = new ListBlobsOptions().setPrefix(config.getPrefix() + path);
+        //MAX_VALUE below represents practically no timeout
+        PagedIterable<BlobItem> blobItems =
+                blobContainerClient.listBlobs(listBlobsOptions, Duration.ofDays(Long.MAX_VALUE));
+        return blobItems.stream().findAny().isEmpty();
+    }
+
+    @Override
+    public IParallelDownloader createParallelDownloader(String bucket, IOManager ioManager) {
+        return new AzureParallelDownloader(ioManager, blobContainerClient, profiler, config);
+    }
+
+    @Override
+    public JsonNode listAsJson(ObjectMapper objectMapper, String bucket) {
+        profiler.objectsList();
+        PagedIterable<BlobItem> blobItems = getBlobItems(bucket, BUCKET_ROOT_PATH);
+        List<BlobItem> blobs = blobItems.stream().distinct().collect(Collectors.toList());
+        blobs = sortBlobItemsByName(blobs);
+        return mapBlobItemsToJson(blobs, objectMapper);
+    }
+
+    private List<BlobItem> sortBlobItemsByName(List<BlobItem> blobs) {
+        return blobs.stream()
+                .sorted((blob1, blob2) -> String.CASE_INSENSITIVE_ORDER.compare(blob1.getName(), blob2.getName()))
+                .collect(Collectors.toList());
+    }
+
+    private ArrayNode mapBlobItemsToJson(List<BlobItem> blobs, ObjectMapper objectMapper) {
+        ArrayNode objectsInfo = objectMapper.createArrayNode();
+        for (BlobItem blob : blobs) {
+            ObjectNode objectInfo = objectsInfo.addObject();
+            objectInfo.put("path", blob.getName());
+            objectInfo.put("size", blob.getProperties().getContentLength());
+        }
+        return objectsInfo;
+    }
+
+    @Override
+    public void close() {
+        // Closing Azure Blob Clients is not required as the underlying netty connection pool
+        // handles the same for the apps.
+        // Ref: https://github.com/Azure/azure-sdk-for-java/issues/17903
+        // Hence this implementation is a no op.
+    }
+
+    private static BlobContainerClient buildClient(AzBlobStorageClientConfig config) {
+        BlobContainerClientBuilder blobContainerClientBuilder =
+                new BlobContainerClientBuilder().containerName(config.getBucket()).endpoint(getEndpoint(config));
+        configCredentialsToAzClient(blobContainerClientBuilder, config);
+        BlobContainerClient blobContainerClient = blobContainerClientBuilder.buildClient();
+        blobContainerClient.createIfNotExists();
+        return blobContainerClient;
+    }
+
+    private static void configCredentialsToAzClient(BlobContainerClientBuilder builder,
+            AzBlobStorageClientConfig config) {
+        if (config.isAnonymousAuth()) {
+            StorageSharedKeyCredential creds =
+                    new StorageSharedKeyCredential(AZURITE_ACCOUNT_NAME, AZURITE_ACCOUNT_KEY);
+            builder.credential(creds);
+        } else {
+            builder.credential(config.createCredentialsProvider());
+        }
+    }
+
+    private static String getEndpoint(AzBlobStorageClientConfig config) {
+        return config.isAnonymousAuth() ? AZURITE_ENDPOINT + config.getBucket()
+                : config.getEndpoint() + "/" + config.getBucket();
+    }
+
+    private List<String> getBlobURLs(Set<BlobItem> blobs) {
+        final String blobURLPrefix = blobContainerClient.getBlobContainerUrl() + "/";
+        return blobs.stream().map(BlobItem::getName).map(blobName -> blobURLPrefix + blobName)
+                .collect(Collectors.toList());
+    }
+}
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzureParallelDownloader.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzureParallelDownloader.java
new file mode 100644
index 0000000..4980587
--- /dev/null
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzureParallelDownloader.java
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.cloud.clients.azure.blobstorage;
+
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.InvalidPathException;
+import java.nio.file.Path;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.asterix.cloud.clients.IParallelDownloader;
+import org.apache.asterix.cloud.clients.profiler.IRequestProfilerLimiter;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.control.nc.io.IOManager;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import com.azure.core.http.rest.PagedIterable;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.models.BlobItem;
+import com.azure.storage.blob.models.ListBlobsOptions;
+
+public class AzureParallelDownloader implements IParallelDownloader {
+    public static final String STORAGE_SUB_DIR = "storage";
+    private final IOManager ioManager;
+    private final BlobContainerClient blobContainerClient;
+    private final IRequestProfilerLimiter profiler;
+    private final AzBlobStorageClientConfig config;
+    private static final Logger LOGGER = LogManager.getLogger();
+
+    public AzureParallelDownloader(IOManager ioManager, BlobContainerClient blobContainerClient,
+            IRequestProfilerLimiter profiler, AzBlobStorageClientConfig config) {
+        this.ioManager = ioManager;
+        this.blobContainerClient = blobContainerClient;
+        this.profiler = profiler;
+        this.config = config;
+    }
+
+    @Override
+    public void downloadFiles(Collection<FileReference> toDownload) throws HyracksDataException {
+        for (FileReference fileReference : toDownload) {
+            BlobClient blobClient =
+                    blobContainerClient.getBlobClient(config.getPrefix() + fileReference.getRelativePath());
+            Path absPath = Path.of(fileReference.getAbsolutePath());
+            Path parentPath = absPath.getParent();
+            OutputStream fileOutputStream = null;
+            try {
+                createDirectories(parentPath);
+                fileOutputStream = Files.newOutputStream(absPath);
+                blobClient.downloadStream(fileOutputStream);
+                fileOutputStream.close();
+            } catch (IOException e) {
+                throw HyracksDataException.create(e);
+            } finally {
+                closeOutputStream(fileOutputStream);
+            }
+        }
+    }
+
+    private static void closeOutputStream(OutputStream fileOutputStream) throws HyracksDataException {
+        if (fileOutputStream != null) {
+            try {
+                fileOutputStream.close();
+            } catch (IOException e) {
+                throw HyracksDataException.create(e);
+            }
+        }
+    }
+
+    @Override
+    public Collection<FileReference> downloadDirectories(Collection<FileReference> directories)
+            throws HyracksDataException {
+        Set<FileReference> failedFiles = new HashSet<>();
+        for (FileReference directory : directories) {
+            PagedIterable<BlobItem> blobsInDir = getBlobItems(directory);
+            for (BlobItem blobItem : blobsInDir) {
+                profiler.objectGet();
+                download(blobItem, failedFiles);
+            }
+        }
+        return failedFiles;
+    }
+
+    private void download(BlobItem blobItem, Set<FileReference> failedFiles) throws HyracksDataException {
+        BlobClient blobClient = blobContainerClient.getBlobClient(blobItem.getName());
+        FileReference diskDestFile = ioManager.resolve(createDiskSubPath(blobItem.getName()));
+        Path absDiskBlobPath = getDiskDestPath(diskDestFile);
+        Path parentDiskPath = absDiskBlobPath.getParent();
+        createDirectories(parentDiskPath);
+        FileOutputStream outputStreamToDest = getOutputStreamToDest(diskDestFile);
+        try {
+            blobClient.downloadStream(outputStreamToDest);
+        } catch (Exception e) {
+            FileReference failedFile = ioManager.resolve(blobItem.getName());
+            failedFiles.add(failedFile);
+        }
+    }
+
+    private String createDiskSubPath(String blobName) {
+        if (!blobName.startsWith(STORAGE_SUB_DIR)) {
+            blobName = blobName.substring(blobName.indexOf(STORAGE_SUB_DIR));
+        }
+        return blobName;
+    }
+
+    private FileOutputStream getOutputStreamToDest(FileReference destFile) throws HyracksDataException {
+        try {
+            return new FileOutputStream(destFile.getAbsolutePath());
+        } catch (FileNotFoundException ex) {
+            throw HyracksDataException.create(ex);
+        }
+    }
+
+    private void createDirectories(Path parentPath) throws HyracksDataException {
+        if (Files.notExists(parentPath))
+            try {
+                Files.createDirectories(parentPath);
+            } catch (IOException ex) {
+                throw HyracksDataException.create(ex);
+            }
+    }
+
+    private Path getDiskDestPath(FileReference destFile) throws HyracksDataException {
+        try {
+            return Path.of(destFile.getAbsolutePath());
+        } catch (InvalidPathException ex) {
+            throw HyracksDataException.create(ex);
+        }
+    }
+
+    private PagedIterable<BlobItem> getBlobItems(FileReference directoryToDownload) {
+        ListBlobsOptions listBlobsOptions =
+                new ListBlobsOptions().setPrefix(config.getPrefix() + directoryToDownload.getRelativePath());
+        return blobContainerClient.listBlobs(listBlobsOptions, null);
+    }
+
+    @Override
+    public void close() {
+        // Closing Azure Blob Clients is not required as the underlying netty connection pool
+        // handles the same for the apps.
+        // Ref: https://github.com/Azure/azure-sdk-for-java/issues/17903
+        // Hence this implementation is a no op.
+    }
+}
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzureRequestRateLimiter.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzureRequestRateLimiter.java
new file mode 100644
index 0000000..c97f3e6
--- /dev/null
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/azure/blobstorage/AzureRequestRateLimiter.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.cloud.clients.azure.blobstorage;
+
+import org.apache.asterix.cloud.clients.AbstractCloudRequestRateLimiter;
+
+public final class AzureRequestRateLimiter extends AbstractCloudRequestRateLimiter {
+
+    public AzureRequestRateLimiter(AzBlobStorageClientConfig config) {
+        super(config.getWriteMaxRequestsPerSeconds(), config.getReadMaxRequestsPerSeconds(),
+                config.getTokenAcquireTimeout());
+    }
+}
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/google/gcs/GCSClientConfig.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/google/gcs/GCSClientConfig.java
index 4edb7a7..e4e471d 100644
--- a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/google/gcs/GCSClientConfig.java
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/google/gcs/GCSClientConfig.java
@@ -19,52 +19,68 @@
 package org.apache.asterix.cloud.clients.google.gcs;
 
 import static org.apache.asterix.external.util.google.gcs.GCSConstants.ENDPOINT_FIELD_NAME;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.STORAGE_PREFIX;
 
 import java.io.IOException;
 import java.util.Map;
 
 import org.apache.asterix.common.config.CloudProperties;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.util.StorageUtil;
 
 import com.google.auth.oauth2.GoogleCredentials;
 import com.google.auth.oauth2.OAuth2Credentials;
 import com.google.cloud.NoCredentials;
 
 public class GCSClientConfig {
-    public static final int WRITE_BUFFER_SIZE = StorageUtil.getIntSizeInBytes(1, StorageUtil.StorageUnit.MEGABYTE);
+
     // The maximum number of files that can be deleted (GCS restriction): https://cloud.google.com/storage/quotas#json-requests
     static final int DELETE_BATCH_SIZE = 100;
     private final String region;
     private final String endpoint;
-    private final String prefix;
     private final boolean anonymousAuth;
     private final long profilerLogInterval;
+    private final long tokenAcquireTimeout;
+    private final int readMaxRequestsPerSeconds;
+    private final int writeMaxRequestsPerSeconds;
+    private final int writeBufferSize;
+    private final String prefix;
 
-    public GCSClientConfig(String region, String endpoint, String prefix, boolean anonymousAuth,
-            long profilerLogInterval) {
+    private GCSClientConfig(String region, String endpoint, boolean anonymousAuth, long profilerLogInterval,
+            long tokenAcquireTimeout, int writeMaxRequestsPerSeconds, int readMaxRequestsPerSeconds,
+            int writeBufferSize, String prefix) {
         this.region = region;
         this.endpoint = endpoint;
-        this.prefix = prefix;
         this.anonymousAuth = anonymousAuth;
         this.profilerLogInterval = profilerLogInterval;
+        this.tokenAcquireTimeout = tokenAcquireTimeout;
+        this.writeMaxRequestsPerSeconds = writeMaxRequestsPerSeconds;
+        this.readMaxRequestsPerSeconds = readMaxRequestsPerSeconds;
+        this.writeBufferSize = writeBufferSize;
+        this.prefix = prefix;
+    }
+
+    public GCSClientConfig(String region, String endpoint, boolean anonymousAuth, long profilerLogInterval,
+            int writeBufferSize, String prefix) {
+        this(region, endpoint, anonymousAuth, profilerLogInterval, 1, 0, 0, writeBufferSize, prefix);
     }
 
     public static GCSClientConfig of(CloudProperties cloudProperties) {
         return new GCSClientConfig(cloudProperties.getStorageRegion(), cloudProperties.getStorageEndpoint(),
-                cloudProperties.getStoragePrefix(), cloudProperties.isStorageAnonymousAuth(),
-                cloudProperties.getProfilerLogInterval());
+                cloudProperties.isStorageAnonymousAuth(), cloudProperties.getProfilerLogInterval(),
+                cloudProperties.getTokenAcquireTimeout(), cloudProperties.getWriteMaxRequestsPerSecond(),
+                cloudProperties.getReadMaxRequestsPerSecond(), cloudProperties.getWriteBufferSize(),
+                cloudProperties.getStoragePrefix());
     }
 
-    public static GCSClientConfig of(Map<String, String> configuration) {
+    public static GCSClientConfig of(Map<String, String> configuration, int writeBufferSize) {
         String endPoint = configuration.getOrDefault(ENDPOINT_FIELD_NAME, "");
         long profilerLogInterval = 0;
 
         String region = "";
-        String prefix = "";
+        String prefix = configuration.getOrDefault(STORAGE_PREFIX, "");
         boolean anonymousAuth = false;
 
-        return new GCSClientConfig(region, endPoint, prefix, anonymousAuth, profilerLogInterval);
+        return new GCSClientConfig(region, endPoint, anonymousAuth, profilerLogInterval, writeBufferSize, prefix);
     }
 
     public String getRegion() {
@@ -75,10 +91,6 @@
         return endpoint;
     }
 
-    public String getPrefix() {
-        return prefix;
-    }
-
     public long getProfilerLogInterval() {
         return profilerLogInterval;
     }
@@ -94,4 +106,24 @@
             throw HyracksDataException.create(e);
         }
     }
+
+    public long getTokenAcquireTimeout() {
+        return tokenAcquireTimeout;
+    }
+
+    public int getWriteMaxRequestsPerSeconds() {
+        return writeMaxRequestsPerSeconds;
+    }
+
+    public int getReadMaxRequestsPerSeconds() {
+        return readMaxRequestsPerSeconds;
+    }
+
+    public int getWriteBufferSize() {
+        return writeBufferSize;
+    }
+
+    public String getPrefix() {
+        return prefix;
+    }
 }
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/google/gcs/GCSCloudClient.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/google/gcs/GCSCloudClient.java
index de242bd..62ca4ec 100644
--- a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/google/gcs/GCSCloudClient.java
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/google/gcs/GCSCloudClient.java
@@ -40,10 +40,10 @@
 import org.apache.asterix.cloud.clients.IParallelDownloader;
 import org.apache.asterix.cloud.clients.profiler.CountRequestProfilerLimiter;
 import org.apache.asterix.cloud.clients.profiler.IRequestProfilerLimiter;
-import org.apache.asterix.cloud.clients.profiler.NoOpRequestProfilerLimiter;
-import org.apache.asterix.cloud.clients.profiler.limiter.NoOpRequestLimiter;
+import org.apache.asterix.cloud.clients.profiler.RequestLimiterNoOpProfiler;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.util.CleanupUtils;
 import org.apache.hyracks.api.util.IoUtil;
 import org.apache.hyracks.control.nc.io.IOManager;
 
@@ -68,16 +68,19 @@
     private final GCSClientConfig config;
     private final ICloudGuardian guardian;
     private final IRequestProfilerLimiter profilerLimiter;
+    private final int writeBufferSize;
 
     public GCSCloudClient(GCSClientConfig config, Storage gcsClient, ICloudGuardian guardian) {
         this.gcsClient = gcsClient;
         this.config = config;
         this.guardian = guardian;
+        this.writeBufferSize = config.getWriteBufferSize();
         long profilerInterval = config.getProfilerLogInterval();
+        GCSRequestRateLimiter limiter = new GCSRequestRateLimiter(config);
         if (profilerInterval > 0) {
-            profilerLimiter = new CountRequestProfilerLimiter(profilerInterval, NoOpRequestLimiter.INSTANCE);
+            profilerLimiter = new CountRequestProfilerLimiter(profilerInterval, limiter);
         } else {
-            profilerLimiter = NoOpRequestProfilerLimiter.INSTANCE;
+            profilerLimiter = new RequestLimiterNoOpProfiler(limiter);
         }
         guardian.setCloudClient(this);
     }
@@ -88,7 +91,7 @@
 
     @Override
     public int getWriteBufferSize() {
-        return GCSClientConfig.WRITE_BUFFER_SIZE;
+        return writeBufferSize;
     }
 
     @Override
@@ -98,20 +101,20 @@
 
     @Override
     public ICloudWriter createWriter(String bucket, String path, IWriteBufferProvider bufferProvider) {
-        return new GCSWriter(bucket, path, gcsClient, profilerLimiter);
+        return new GCSWriter(bucket, config.getPrefix() + path, gcsClient, profilerLimiter, guardian, writeBufferSize);
     }
 
     @Override
     public Set<CloudFile> listObjects(String bucket, String path, FilenameFilter filter) {
         guardian.checkReadAccess(bucket, path);
         profilerLimiter.objectsList();
-        Page<Blob> blobs =
-                gcsClient.list(bucket, BlobListOption.prefix(path), BlobListOption.fields(Storage.BlobField.SIZE));
+        Page<Blob> blobs = gcsClient.list(bucket, BlobListOption.prefix(config.getPrefix() + path),
+                BlobListOption.fields(Storage.BlobField.SIZE));
 
         Set<CloudFile> files = new HashSet<>();
         for (Blob blob : blobs.iterateAll()) {
             if (filter.accept(null, IoUtil.getFileNameFromPath(blob.getName()))) {
-                files.add(CloudFile.of(blob.getName(), blob.getSize()));
+                files.add(CloudFile.of(stripCloudPrefix(blob.getName()), blob.getSize()));
             }
         }
         return files;
@@ -119,8 +122,9 @@
 
     @Override
     public int read(String bucket, String path, long offset, ByteBuffer buffer) throws HyracksDataException {
+        guardian.checkReadAccess(bucket, path);
         profilerLimiter.objectGet();
-        BlobId blobId = BlobId.of(bucket, path);
+        BlobId blobId = BlobId.of(bucket, config.getPrefix() + path);
         long readTo = offset + buffer.remaining();
         int totalRead = 0;
         try (ReadChannel from = gcsClient.reader(blobId).limit(readTo)) {
@@ -140,8 +144,9 @@
 
     @Override
     public byte[] readAllBytes(String bucket, String path) {
+        guardian.checkReadAccess(bucket, path);
         profilerLimiter.objectGet();
-        BlobId blobId = BlobId.of(bucket, path);
+        BlobId blobId = BlobId.of(bucket, config.getPrefix() + path);
         try {
             return gcsClient.readAllBytes(blobId);
         } catch (StorageException e) {
@@ -151,12 +156,15 @@
 
     @Override
     public InputStream getObjectStream(String bucket, String path, long offset, long length) {
+        guardian.checkReadAccess(bucket, path);
         profilerLimiter.objectGet();
-        try (ReadChannel reader = gcsClient.reader(bucket, path).limit(offset + length)) {
+        ReadChannel reader = null;
+        try {
+            reader = gcsClient.reader(bucket, config.getPrefix() + path).limit(offset + length);
             reader.seek(offset);
             return Channels.newInputStream(reader);
-        } catch (StorageException | IOException e) {
-            throw new IllegalStateException(e);
+        } catch (StorageException | IOException ex) {
+            throw new RuntimeException(CleanupUtils.close(reader, ex));
         }
     }
 
@@ -164,14 +172,15 @@
     public void write(String bucket, String path, byte[] data) {
         guardian.checkWriteAccess(bucket, path);
         profilerLimiter.objectWrite();
-        BlobInfo blobInfo = BlobInfo.newBuilder(bucket, path).build();
+        BlobInfo blobInfo = BlobInfo.newBuilder(bucket, config.getPrefix() + path).build();
         gcsClient.create(blobInfo, data);
     }
 
     @Override
     public void copy(String bucket, String srcPath, FileReference destPath) {
-        Page<Blob> blobs = gcsClient.list(bucket, BlobListOption.prefix(srcPath));
+        guardian.checkReadAccess(bucket, srcPath);
         profilerLimiter.objectsList();
+        Page<Blob> blobs = gcsClient.list(bucket, BlobListOption.prefix(config.getPrefix() + srcPath));
         for (Blob blob : blobs.iterateAll()) {
             profilerLimiter.objectCopy();
             BlobId source = blob.getBlobId();
@@ -194,7 +203,7 @@
         while (pathIter.hasNext()) {
             batchRequest = gcsClient.batch();
             for (int i = 0; pathIter.hasNext() && i < DELETE_BATCH_SIZE; i++) {
-                BlobId blobId = BlobId.of(bucket, pathIter.next());
+                BlobId blobId = BlobId.of(bucket, config.getPrefix() + pathIter.next());
                 guardian.checkWriteAccess(bucket, blobId.getName());
                 batchRequest.delete(blobId);
             }
@@ -208,7 +217,8 @@
     public long getObjectSize(String bucket, String path) {
         guardian.checkReadAccess(bucket, path);
         profilerLimiter.objectGet();
-        Blob blob = gcsClient.get(bucket, path, Storage.BlobGetOption.fields(Storage.BlobField.SIZE));
+        Blob blob =
+                gcsClient.get(bucket, config.getPrefix() + path, Storage.BlobGetOption.fields(Storage.BlobField.SIZE));
         if (blob == null) {
             return 0;
         }
@@ -219,7 +229,8 @@
     public boolean exists(String bucket, String path) {
         guardian.checkReadAccess(bucket, path);
         profilerLimiter.objectGet();
-        Blob blob = gcsClient.get(bucket, path, Storage.BlobGetOption.fields(Storage.BlobField.values()));
+        Blob blob = gcsClient.get(bucket, config.getPrefix() + path,
+                Storage.BlobGetOption.fields(Storage.BlobField.values()));
         return blob != null && blob.exists();
     }
 
@@ -227,7 +238,7 @@
     public boolean isEmptyPrefix(String bucket, String path) {
         guardian.checkReadAccess(bucket, path);
         profilerLimiter.objectsList();
-        Page<Blob> blobs = gcsClient.list(bucket, BlobListOption.prefix(path));
+        Page<Blob> blobs = gcsClient.list(bucket, BlobListOption.prefix(config.getPrefix() + path));
         return !blobs.hasNextPage();
     }
 
@@ -272,4 +283,8 @@
         }
         return builder.build().getService();
     }
-}
+
+    private String stripCloudPrefix(String objectName) {
+        return objectName.substring(config.getPrefix().length());
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/google/gcs/GCSParallelDownloader.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/google/gcs/GCSParallelDownloader.java
index 0994cea..0d30120 100644
--- a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/google/gcs/GCSParallelDownloader.java
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/google/gcs/GCSParallelDownloader.java
@@ -56,6 +56,7 @@
     private final Storage gcsClient;
     private final TransferManager transferManager;
     private final IRequestProfilerLimiter profiler;
+    private final GCSClientConfig config;
 
     public GCSParallelDownloader(String bucket, IOManager ioManager, GCSClientConfig config,
             IRequestProfilerLimiter profiler) throws HyracksDataException {
@@ -70,18 +71,21 @@
         this.gcsClient = builder.build().getService();
         this.transferManager =
                 TransferManagerConfig.newBuilder().setStorageOptions(builder.build()).build().getService();
+        this.config = config;
     }
 
     @Override
     public void downloadFiles(Collection<FileReference> toDownload) throws HyracksDataException {
-        ParallelDownloadConfig.Builder config = ParallelDownloadConfig.newBuilder().setBucketName(bucket);
+        ParallelDownloadConfig.Builder downConfig =
+                ParallelDownloadConfig.newBuilder().setBucketName(bucket).setStripPrefix(this.config.getPrefix());
+
         Map<Path, List<BlobInfo>> pathListMap = new HashMap<>();
         try {
             for (FileReference fileReference : toDownload) {
                 profiler.objectGet();
                 FileUtils.createParentDirectories(fileReference.getFile());
-                addToMap(pathListMap, fileReference.getDeviceHandle().getMount().toPath(),
-                        BlobInfo.newBuilder(BlobId.of(bucket, fileReference.getRelativePath())).build());
+                addToMap(pathListMap, fileReference.getDeviceHandle().getMount().toPath(), BlobInfo
+                        .newBuilder(BlobId.of(bucket, config.getPrefix() + fileReference.getRelativePath())).build());
             }
         } catch (IOException e) {
             throw HyracksDataException.create(e);
@@ -89,7 +93,7 @@
         List<DownloadJob> downloadJobs = new ArrayList<>(pathListMap.size());
         for (Map.Entry<Path, List<BlobInfo>> entry : pathListMap.entrySet()) {
             downloadJobs.add(transferManager.downloadBlobs(entry.getValue(),
-                    config.setDownloadDirectory(entry.getKey()).build()));
+                    downConfig.setDownloadDirectory(entry.getKey()).build()));
         }
         downloadJobs.forEach(DownloadJob::getDownloadResults);
     }
@@ -98,20 +102,22 @@
     public Collection<FileReference> downloadDirectories(Collection<FileReference> toDownload)
             throws HyracksDataException {
         Set<FileReference> failedFiles = new HashSet<>();
-        ParallelDownloadConfig.Builder config = ParallelDownloadConfig.newBuilder().setBucketName(bucket);
+        ParallelDownloadConfig.Builder config =
+                ParallelDownloadConfig.newBuilder().setBucketName(bucket).setStripPrefix(this.config.getPrefix());
 
         Map<Path, List<BlobInfo>> pathListMap = new HashMap<>();
         for (FileReference fileReference : toDownload) {
             profiler.objectMultipartDownload();
-            Page<Blob> blobs = gcsClient.list(bucket, Storage.BlobListOption.prefix(fileReference.getRelativePath()));
+            Page<Blob> blobs = gcsClient.list(bucket,
+                    Storage.BlobListOption.prefix(this.config.getPrefix() + fileReference.getRelativePath()));
             for (Blob blob : blobs.iterateAll()) {
                 addToMap(pathListMap, fileReference.getDeviceHandle().getMount().toPath(), blob.asBlobInfo());
             }
         }
         List<DownloadJob> downloadJobs = new ArrayList<>(pathListMap.size());
         for (Map.Entry<Path, List<BlobInfo>> entry : pathListMap.entrySet()) {
-            downloadJobs.add(transferManager.downloadBlobs(entry.getValue(),
-                    config.setDownloadDirectory(entry.getKey()).build()));
+            ParallelDownloadConfig parallelDownloadConfig = config.setDownloadDirectory(entry.getKey()).build();
+            downloadJobs.add(transferManager.downloadBlobs(entry.getValue(), parallelDownloadConfig));
         }
         List<DownloadResult> results;
         for (DownloadJob job : downloadJobs) {
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/google/gcs/GCSRequestRateLimiter.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/google/gcs/GCSRequestRateLimiter.java
new file mode 100644
index 0000000..a68c3f9
--- /dev/null
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/google/gcs/GCSRequestRateLimiter.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.cloud.clients.google.gcs;
+
+import org.apache.asterix.cloud.clients.AbstractCloudRequestRateLimiter;
+
+public class GCSRequestRateLimiter extends AbstractCloudRequestRateLimiter {
+
+    public GCSRequestRateLimiter(GCSClientConfig config) {
+        super(config.getWriteMaxRequestsPerSeconds(), config.getReadMaxRequestsPerSeconds(),
+                config.getTokenAcquireTimeout());
+    }
+}
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/google/gcs/GCSWriter.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/google/gcs/GCSWriter.java
index 41d1a71..8d68f01 100644
--- a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/google/gcs/GCSWriter.java
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/google/gcs/GCSWriter.java
@@ -18,11 +18,10 @@
  */
 package org.apache.asterix.cloud.clients.google.gcs;
 
-import static org.apache.asterix.cloud.clients.google.gcs.GCSClientConfig.WRITE_BUFFER_SIZE;
-
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
+import org.apache.asterix.cloud.clients.ICloudGuardian;
 import org.apache.asterix.cloud.clients.ICloudWriter;
 import org.apache.asterix.cloud.clients.profiler.IRequestProfilerLimiter;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -40,14 +39,20 @@
     private final String path;
     private final IRequestProfilerLimiter profiler;
     private final Storage gcsClient;
+    private final ICloudGuardian guardian;
+    private final int writeBufferSize;
+
     private WriteChannel writer = null;
     private long writtenBytes;
 
-    public GCSWriter(String bucket, String path, Storage gcsClient, IRequestProfilerLimiter profiler) {
+    public GCSWriter(String bucket, String path, Storage gcsClient, IRequestProfilerLimiter profiler,
+            ICloudGuardian guardian, int writeBufferSize) {
         this.bucket = bucket;
         this.path = path;
         this.profiler = profiler;
         this.gcsClient = gcsClient;
+        this.guardian = guardian;
+        this.writeBufferSize = writeBufferSize;
         writtenBytes = 0;
     }
 
@@ -58,6 +63,7 @@
 
     @Override
     public int write(ByteBuffer page) throws HyracksDataException {
+        guardian.checkIsolatedWriteAccess(bucket, path);
         profiler.objectMultipartUpload();
         setUploadId();
         int written = 0;
@@ -93,6 +99,7 @@
 
     @Override
     public void finish() throws HyracksDataException {
+        guardian.checkWriteAccess(bucket, path);
         setUploadId();
         profiler.objectMultipartUpload();
         try {
@@ -115,7 +122,7 @@
     private void setUploadId() {
         if (writer == null) {
             writer = gcsClient.writer(BlobInfo.newBuilder(BlobId.of(bucket, path)).build());
-            writer.setChunkSize(WRITE_BUFFER_SIZE);
+            writer.setChunkSize(writeBufferSize);
             writtenBytes = 0;
             log("STARTED");
         }
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/CountRequestProfilerLimiter.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/CountRequestProfilerLimiter.java
index 16ffb7a..95383f6 100644
--- a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/CountRequestProfilerLimiter.java
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/CountRequestProfilerLimiter.java
@@ -140,6 +140,16 @@
         return multipartDownloadCounter.get();
     }
 
+    @Override
+    public long getReadThrottleCount() {
+        return limiter.getReadThrottleCount();
+    }
+
+    @Override
+    public long getWriteThrottleCount() {
+        return limiter.getWriteThrottleCount();
+    }
+
     private void log() {
         if (LOGGER.isEnabled(LOG_LEVEL)) {
             long currentTime = System.nanoTime();
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/IRequestProfilerLimiter.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/IRequestProfilerLimiter.java
index b86cd48..fc0cbef 100644
--- a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/IRequestProfilerLimiter.java
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/IRequestProfilerLimiter.java
@@ -47,4 +47,7 @@
 
     long objectMultipartDownloadCount();
 
+    long getReadThrottleCount();
+
+    long getWriteThrottleCount();
 }
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/NoOpRequestProfilerLimiter.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/NoOpRequestProfilerLimiter.java
index ab658f5..832457b 100644
--- a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/NoOpRequestProfilerLimiter.java
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/NoOpRequestProfilerLimiter.java
@@ -93,4 +93,14 @@
     public long objectMultipartDownloadCount() {
         return 0;
     }
+
+    @Override
+    public long getReadThrottleCount() {
+        return 0;
+    }
+
+    @Override
+    public long getWriteThrottleCount() {
+        return 0;
+    }
 }
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/RequestLimiterNoOpProfiler.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/RequestLimiterNoOpProfiler.java
index cce2f8e..9d5118a 100644
--- a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/RequestLimiterNoOpProfiler.java
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/RequestLimiterNoOpProfiler.java
@@ -96,4 +96,14 @@
     public long objectMultipartDownloadCount() {
         return 0;
     }
+
+    @Override
+    public long getReadThrottleCount() {
+        return limiter.getReadThrottleCount();
+    }
+
+    @Override
+    public long getWriteThrottleCount() {
+        return limiter.getWriteThrottleCount();
+    }
 }
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/IRateLimiter.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/IRateLimiter.java
index 741735b..883990d 100644
--- a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/IRateLimiter.java
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/IRateLimiter.java
@@ -26,4 +26,11 @@
      * Acquire permit or wait if rate limit exceeded
      */
     void acquire();
+
+    /**
+     * Get the number of throttled requests
+     *
+     * @return the number of throttled requests
+     */
+    long getThrottleCount();
 }
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/IRequestRateLimiter.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/IRequestRateLimiter.java
index 98b2eab..bc5bdbd 100644
--- a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/IRequestRateLimiter.java
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/IRequestRateLimiter.java
@@ -37,4 +37,18 @@
      * Perform a list request
      */
     void listRequest();
+
+    /**
+     * Get the number of throttled read requests
+     *
+     * @return the number of throttled read requests
+     */
+    long getReadThrottleCount();
+
+    /**
+     * Get the number of throttled write requests
+     *
+     * @return the number of throttled write requests
+     */
+    long getWriteThrottleCount();
 }
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/NoOpRateLimiter.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/NoOpRateLimiter.java
index 4dd8c88..356538e 100644
--- a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/NoOpRateLimiter.java
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/NoOpRateLimiter.java
@@ -28,4 +28,9 @@
     public void acquire() {
         // NoOp
     }
+
+    @Override
+    public long getThrottleCount() {
+        return 0;
+    }
 }
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/NoOpRequestLimiter.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/NoOpRequestLimiter.java
index ea89a2e..182d925 100644
--- a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/NoOpRequestLimiter.java
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/NoOpRequestLimiter.java
@@ -38,4 +38,14 @@
     public void listRequest() {
         // NoOp
     }
+
+    @Override
+    public long getReadThrottleCount() {
+        return 0;
+    }
+
+    @Override
+    public long getWriteThrottleCount() {
+        return 0;
+    }
 }
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/TokenBasedRateLimiter.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/TokenBasedRateLimiter.java
index a0273fb..564e1c2 100644
--- a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/TokenBasedRateLimiter.java
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/clients/profiler/limiter/TokenBasedRateLimiter.java
@@ -28,6 +28,7 @@
 public final class TokenBasedRateLimiter implements IRateLimiter {
     private static final Logger LOGGER = LogManager.getLogger();
     private static final long SECOND_NANO = TimeUnit.SECONDS.toNanos(1);
+    private final AtomicLong throttleCount = new AtomicLong();
     private final long acquireTimeoutNano;
     private final int maxTokensPerSecond;
     private final Semaphore semaphore;
@@ -54,6 +55,7 @@
                 if (semaphore.tryAcquire(acquireTimeoutNano, TimeUnit.NANOSECONDS)) {
                     return;
                 }
+                throttleCount.incrementAndGet();
             } catch (InterruptedException e) {
                 Thread.currentThread().interrupt();
                 LOGGER.debug("Interrupted while waiting for acquiring a request token", e);
@@ -62,6 +64,11 @@
         }
     }
 
+    @Override
+    public long getThrottleCount() {
+        return throttleCount.get();
+    }
+
     private void refillTokens() {
         long refillTime = lastRefillTime.get();
         long now = System.nanoTime();
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/lazy/filesystem/HolePuncherProvider.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/lazy/filesystem/HolePuncherProvider.java
index 91de5ae..e50559f 100644
--- a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/lazy/filesystem/HolePuncherProvider.java
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/lazy/filesystem/HolePuncherProvider.java
@@ -52,7 +52,7 @@
 
         // Running a debug hole puncher on a non-Linux box
         String osName = FileSystemOperationDispatcherUtil.getOSName();
-        LOGGER.warn("Using 'DebugHolePuncher' as the OS '{}' does not support punishing holes", osName);
+        LOGGER.warn("Using 'DebugHolePuncher' as the OS '{}' does not support punching holes", osName);
         return new DebugHolePuncher(cloudIOManager, bufferProvider);
     }
 
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/parquet/ParquetExternalWriterFactory.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/parquet/ParquetExternalWriterFactory.java
new file mode 100644
index 0000000..d754068
--- /dev/null
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/parquet/ParquetExternalWriterFactory.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.cloud.parquet;
+
+import static org.apache.asterix.external.writer.printer.parquet.ParquetSchemaLazyVisitor.generateSchema;
+
+import java.io.Serializable;
+
+import org.apache.asterix.external.writer.printer.ParquetExternalFilePrinterFactory;
+import org.apache.asterix.external.writer.printer.parquet.ParquetSchemaTree;
+import org.apache.asterix.runtime.writer.ExternalFileWriter;
+import org.apache.asterix.runtime.writer.IExternalFileWriter;
+import org.apache.asterix.runtime.writer.IExternalFileWriterFactory;
+import org.apache.asterix.runtime.writer.IPathResolver;
+import org.apache.hyracks.algebricks.runtime.writers.IExternalWriter;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.parquet.schema.MessageType;
+
+public class ParquetExternalWriterFactory implements Serializable {
+
+    private static final long serialVersionUID = 8971234908711236L;
+    private final IExternalFileWriterFactory writerFactory;
+    private final int maxResult;
+    private final ParquetExternalFilePrinterFactory printerFactory;
+    private final IPathResolver resolver;
+    private final IHyracksTaskContext ctx;
+
+    public ParquetExternalWriterFactory(IHyracksTaskContext ctx, IExternalFileWriterFactory writerFactory,
+            int maxResult, ParquetExternalFilePrinterFactory printerFactory, IPathResolver resolver) {
+        this.ctx = ctx;
+        this.writerFactory = writerFactory;
+        this.maxResult = maxResult;
+        this.printerFactory = printerFactory;
+        this.resolver = resolver;
+    }
+
+    public IExternalWriter createWriter(ParquetSchemaTree.SchemaNode schemaNode) throws HyracksDataException {
+        MessageType schema = generateSchema(schemaNode);
+        printerFactory.setParquetSchemaString(schema.toString());
+        IExternalFileWriter writer = writerFactory.createWriter(ctx, printerFactory);
+        return new ExternalFileWriter(resolver, writer, maxResult);
+    }
+
+}
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/parquet/ParquetSchemaInferPoolWriter.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/parquet/ParquetSchemaInferPoolWriter.java
new file mode 100644
index 0000000..b500cbe
--- /dev/null
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/parquet/ParquetSchemaInferPoolWriter.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.cloud.parquet;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.external.writer.printer.parquet.ISchemaChecker;
+import org.apache.asterix.external.writer.printer.parquet.ParquetSchemaLazyVisitor;
+import org.apache.asterix.external.writer.printer.parquet.ParquetSchemaTree;
+import org.apache.hyracks.algebricks.runtime.writers.IExternalWriter;
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class ParquetSchemaInferPoolWriter {
+    private final ParquetExternalWriterFactory writerFactory;
+
+    private List<ParquetSchemaTree.SchemaNode> schemaNodes;
+    private List<IExternalWriter> writerList;
+    private final int maxSchemas;
+    private ISchemaChecker schemaChecker;
+    private ParquetSchemaLazyVisitor schemaLazyVisitor;
+
+    public ParquetSchemaInferPoolWriter(ParquetExternalWriterFactory writerFactory, ISchemaChecker schemaChecker,
+            ParquetSchemaLazyVisitor parquetSchemaLazyVisitor, int maxSchemas) {
+        this.writerFactory = writerFactory;
+        this.schemaChecker = schemaChecker;
+        this.schemaLazyVisitor = parquetSchemaLazyVisitor;
+        this.maxSchemas = maxSchemas;
+        this.schemaNodes = new ArrayList<>();
+        this.writerList = new ArrayList<>();
+    }
+
+    public void inferSchema(IValueReference value) throws HyracksDataException {
+        for (int i = 0; i < schemaNodes.size(); i++) {
+            ISchemaChecker.SchemaComparisonType schemaComparisonType =
+                    schemaChecker.checkSchema(schemaNodes.get(i), value);
+
+            if (schemaComparisonType.equals(ISchemaChecker.SchemaComparisonType.EQUIVALENT)) {
+                return;
+            } else if (schemaComparisonType.equals(ISchemaChecker.SchemaComparisonType.GROWING)) {
+                schemaNodes.set(i, schemaLazyVisitor.inferSchema(value));
+                closeWriter(i);
+                return;
+            }
+        }
+
+        if (schemaNodes.size() == maxSchemas) {
+            throw new HyracksDataException(ErrorCode.SCHEMA_LIMIT_EXCEEDED, maxSchemas);
+        }
+        schemaNodes.add(schemaLazyVisitor.inferSchema(value));
+        writerList.add(null);
+    }
+
+    public void initNewPartition(IFrameTupleReference tuple) throws HyracksDataException {
+        for (int i = 0; i < writerList.size(); i++) {
+            createOrInitPartition(i, tuple);
+        }
+    }
+
+    public void write(IValueReference value) throws HyracksDataException {
+        for (int i = 0; i < schemaNodes.size(); i++) {
+            if (schemaChecker.checkSchema(schemaNodes.get(i), value)
+                    .equals(ISchemaChecker.SchemaComparisonType.EQUIVALENT)) {
+                createOrWrite(i, value);
+                return;
+            }
+        }
+    }
+
+    public void close() throws HyracksDataException {
+        for (int i = 0; i < writerList.size(); i++) {
+            closeWriter(i);
+        }
+    }
+
+    private void createOrInitPartition(int index, IFrameTupleReference tupleReference) throws HyracksDataException {
+        if (writerList.get(index) == null) {
+            createWriter(index);
+        }
+        writerList.get(index).initNewPartition(tupleReference);
+    }
+
+    private void createOrWrite(int index, IValueReference value) throws HyracksDataException {
+        if (writerList.get(index) == null) {
+            createWriter(index);
+        }
+        writerList.get(index).write(value);
+    }
+
+    private void createWriter(int index) throws HyracksDataException {
+        writerList.set(index, writerFactory.createWriter(schemaNodes.get(index)));
+        writerList.get(index).open();
+    }
+
+    private void closeWriter(int index) throws HyracksDataException {
+        if (writerList.get(index) != null) {
+            writerList.get(index).close();
+            writerList.set(index, null);
+        }
+    }
+
+}
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/parquet/ParquetSinkExternalWriterFactory.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/parquet/ParquetSinkExternalWriterFactory.java
new file mode 100644
index 0000000..3a276a2
--- /dev/null
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/parquet/ParquetSinkExternalWriterFactory.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.cloud.parquet;
+
+import org.apache.asterix.external.writer.printer.ParquetExternalFilePrinterFactory;
+import org.apache.asterix.external.writer.printer.ParquetExternalFilePrinterFactoryProvider;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.writer.IExternalFileWriterFactory;
+import org.apache.asterix.runtime.writer.IPathResolverFactory;
+import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
+import org.apache.hyracks.algebricks.runtime.operators.base.AbstractPushRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.writer.WriterPartitionerFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class ParquetSinkExternalWriterFactory extends AbstractPushRuntimeFactory {
+    private static final long serialVersionUID = -1285997525553685225L;
+    private final WriterPartitionerFactory partitionerFactory;
+    private final RecordDescriptor inputRecordDesc;
+    private final int sourceColumn;
+    private final int maxSchemas;
+    private final IAType sourceType;
+    private final IExternalFileWriterFactory writerFactory;
+    private final int maxResult;
+    private final ParquetExternalFilePrinterFactoryProvider printerFactoryProvider;
+    private final IPathResolverFactory pathResolverFactory;
+
+    public ParquetSinkExternalWriterFactory(WriterPartitionerFactory partitionerFactory,
+            RecordDescriptor inputRecordDesc, int sourceColumn, IAType sourceType, int maxSchemas,
+            IExternalFileWriterFactory writerFactory, int maxResult,
+            ParquetExternalFilePrinterFactoryProvider printerFactoryProvider,
+            IPathResolverFactory pathResolverFactory) {
+        this.partitionerFactory = partitionerFactory;
+        this.inputRecordDesc = inputRecordDesc;
+        this.sourceColumn = sourceColumn;
+        this.sourceType = sourceType;
+        this.maxSchemas = maxSchemas;
+        this.writerFactory = writerFactory;
+        this.maxResult = maxResult;
+        this.printerFactoryProvider = printerFactoryProvider;
+        this.pathResolverFactory = pathResolverFactory;
+    }
+
+    @Override
+    public IPushRuntime[] createPushRuntime(IHyracksTaskContext ctx) throws HyracksDataException {
+        ParquetExternalFilePrinterFactory printerFactory =
+                (ParquetExternalFilePrinterFactory) printerFactoryProvider.createPrinterFactory();
+        ParquetExternalWriterFactory parquetExternalWriterFactory = new ParquetExternalWriterFactory(ctx, writerFactory,
+                maxResult, printerFactory, pathResolverFactory.createResolver(ctx));
+        ParquetSinkExternalWriterRuntime runtime =
+                new ParquetSinkExternalWriterRuntime(sourceColumn, partitionerFactory.createPartitioner(),
+                        inputRecordDesc, parquetExternalWriterFactory, sourceType, maxSchemas);
+        return new IPushRuntime[] { runtime };
+    }
+}
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/parquet/ParquetSinkExternalWriterRuntime.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/parquet/ParquetSinkExternalWriterRuntime.java
new file mode 100644
index 0000000..3dbd4d3
--- /dev/null
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/parquet/ParquetSinkExternalWriterRuntime.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.cloud.parquet;
+
+import java.nio.ByteBuffer;
+
+import org.apache.asterix.external.writer.printer.parquet.ParquetSchemaLazyVisitor;
+import org.apache.asterix.external.writer.printer.parquet.SchemaCheckerLazyVisitor;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputSinkPushRuntime;
+import org.apache.hyracks.algebricks.runtime.operators.writer.IWriterPartitioner;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class ParquetSinkExternalWriterRuntime extends AbstractOneInputSinkPushRuntime {
+    private final int sourceColumn;
+    private final IWriterPartitioner partitioner;
+    private final IPointable sourceValue;
+    private final ParquetExternalWriterFactory writerFactory;
+    private FrameTupleAccessor tupleAccessor;
+    private FrameTupleReference tupleRef;
+    private IFrameWriter frameWriter;
+    private final int maxSchemas;
+    private final IAType sourceType;
+    private ParquetSchemaInferPoolWriter poolWriter;
+
+    public ParquetSinkExternalWriterRuntime(int sourceColumn, IWriterPartitioner partitioner,
+            RecordDescriptor inputRecordDesc, ParquetExternalWriterFactory writerFactory, IAType sourceType,
+            int maxSchemas) {
+        this.sourceColumn = sourceColumn;
+        this.partitioner = partitioner;
+        this.sourceValue = new VoidPointable();
+        this.inputRecordDesc = inputRecordDesc;
+        this.writerFactory = writerFactory;
+        this.sourceType = sourceType;
+        this.maxSchemas = maxSchemas;
+    }
+
+    @Override
+    public void setOutputFrameWriter(int index, IFrameWriter frameWriter, RecordDescriptor recordDesc) {
+        this.frameWriter = frameWriter;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        if (tupleAccessor == null) {
+            tupleAccessor = new FrameTupleAccessor(inputRecordDesc);
+            tupleRef = new FrameTupleReference();
+        }
+
+        poolWriter = new ParquetSchemaInferPoolWriter(writerFactory, new SchemaCheckerLazyVisitor(sourceType),
+                new ParquetSchemaLazyVisitor(sourceType), maxSchemas);
+        this.frameWriter.open();
+
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        tupleAccessor.reset(buffer);
+
+        for (int i = 0; i < tupleAccessor.getTupleCount(); i++) {
+            tupleRef.reset(tupleAccessor, i);
+            setValue(tupleRef, sourceColumn, sourceValue);
+            poolWriter.inferSchema(sourceValue);
+        }
+
+        for (int i = 0; i < tupleAccessor.getTupleCount(); i++) {
+            tupleRef.reset(tupleAccessor, i);
+            setValue(tupleRef, sourceColumn, sourceValue);
+            if (partitioner.isNewPartition(tupleAccessor, i)) {
+                poolWriter.initNewPartition(tupleRef);
+            }
+            poolWriter.write(sourceValue);
+        }
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+        frameWriter.fail();
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        poolWriter.close();
+        frameWriter.close();
+    }
+
+    private void setValue(IFrameTupleReference tuple, int column, IPointable value) {
+        byte[] data = tuple.getFieldData(column);
+        int start = tuple.getFieldStart(column);
+        int length = tuple.getFieldLength(column);
+        value.set(data, start, length);
+    }
+}
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/writer/AbstractCloudExternalFileWriterFactory.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/writer/AbstractCloudExternalFileWriterFactory.java
index 589ee79..198b3ad 100644
--- a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/writer/AbstractCloudExternalFileWriterFactory.java
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/writer/AbstractCloudExternalFileWriterFactory.java
@@ -30,6 +30,7 @@
 import org.apache.asterix.cloud.WriterSingleBufferProvider;
 import org.apache.asterix.cloud.clients.ICloudClient;
 import org.apache.asterix.cloud.clients.ICloudWriter;
+import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.external.util.ExternalDataConstants;
@@ -60,17 +61,17 @@
         writeBufferSize = externalConfig.getWriteBufferSize();
     }
 
-    abstract ICloudClient createCloudClient() throws CompilationException;
+    abstract ICloudClient createCloudClient(IApplicationContext appCtx) throws CompilationException;
 
     abstract boolean isNoContainerFoundException(IOException e);
 
     abstract boolean isSdkException(Throwable e);
 
-    final void buildClient() throws HyracksDataException {
+    final void buildClient(IApplicationContext appCtx) throws HyracksDataException {
         try {
             synchronized (this) {
                 if (cloudClient == null) {
-                    cloudClient = createCloudClient();
+                    cloudClient = createCloudClient(appCtx);
                 }
             }
         } catch (CompilationException e) {
@@ -79,8 +80,8 @@
     }
 
     @Override
-    public final void validate() throws AlgebricksException {
-        ICloudClient testClient = createCloudClient();
+    public final void validate(IApplicationContext appCtx) throws AlgebricksException {
+        ICloudClient testClient = createCloudClient(appCtx);
         String bucket = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
 
         if (bucket == null || bucket.isEmpty()) {
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/writer/GCSExternalFileWriterFactory.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/writer/GCSExternalFileWriterFactory.java
index 9e9c003..63a8366 100644
--- a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/writer/GCSExternalFileWriterFactory.java
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/writer/GCSExternalFileWriterFactory.java
@@ -24,6 +24,7 @@
 import org.apache.asterix.cloud.clients.ICloudGuardian;
 import org.apache.asterix.cloud.clients.google.gcs.GCSClientConfig;
 import org.apache.asterix.cloud.clients.google.gcs.GCSCloudClient;
+import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.google.gcs.GCSUtils;
@@ -61,8 +62,8 @@
     }
 
     @Override
-    ICloudClient createCloudClient() throws CompilationException {
-        GCSClientConfig config = GCSClientConfig.of(configuration);
+    ICloudClient createCloudClient(IApplicationContext appCtx) throws CompilationException {
+        GCSClientConfig config = GCSClientConfig.of(configuration, writeBufferSize);
         return new GCSCloudClient(config, GCSUtils.buildClient(configuration),
                 ICloudGuardian.NoOpCloudGuardian.INSTANCE);
     }
@@ -80,7 +81,7 @@
     @Override
     public IExternalFileWriter createWriter(IHyracksTaskContext context, IExternalPrinterFactory printerFactory)
             throws HyracksDataException {
-        buildClient();
+        buildClient(((IApplicationContext) context.getJobletContext().getServiceContext().getApplicationContext()));
         String bucket = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
         IExternalPrinter printer = printerFactory.createPrinter();
         IWarningCollector warningCollector = context.getWarningCollector();
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/writer/S3ExternalFileWriterFactory.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/writer/S3ExternalFileWriterFactory.java
index e07acc0..d268efd 100644
--- a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/writer/S3ExternalFileWriterFactory.java
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/writer/S3ExternalFileWriterFactory.java
@@ -24,9 +24,10 @@
 import org.apache.asterix.cloud.clients.ICloudGuardian;
 import org.apache.asterix.cloud.clients.aws.s3.S3ClientConfig;
 import org.apache.asterix.cloud.clients.aws.s3.S3CloudClient;
+import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.aws.s3.S3Utils;
+import org.apache.asterix.external.util.aws.s3.S3AuthUtils;
 import org.apache.asterix.runtime.writer.ExternalFileWriterConfiguration;
 import org.apache.asterix.runtime.writer.IExternalFileWriter;
 import org.apache.asterix.runtime.writer.IExternalFileWriterFactory;
@@ -61,9 +62,9 @@
     }
 
     @Override
-    ICloudClient createCloudClient() throws CompilationException {
+    ICloudClient createCloudClient(IApplicationContext appCtx) throws CompilationException {
         S3ClientConfig config = S3ClientConfig.of(configuration, writeBufferSize);
-        return new S3CloudClient(config, S3Utils.buildAwsS3Client(configuration),
+        return new S3CloudClient(config, S3AuthUtils.buildAwsS3Client(appCtx, configuration),
                 ICloudGuardian.NoOpCloudGuardian.INSTANCE);
     }
 
@@ -80,7 +81,7 @@
     @Override
     public IExternalFileWriter createWriter(IHyracksTaskContext context, IExternalPrinterFactory printerFactory)
             throws HyracksDataException {
-        buildClient();
+        buildClient(((IApplicationContext) context.getJobletContext().getServiceContext().getApplicationContext()));
         String bucket = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
         IExternalPrinter printer = printerFactory.createPrinter();
         IWarningCollector warningCollector = context.getWarningCollector();
diff --git a/asterixdb/asterix-cloud/src/test/java/org/apache/asterix/cloud/azure/LSMAzBlobStorageTest.java b/asterixdb/asterix-cloud/src/test/java/org/apache/asterix/cloud/azure/LSMAzBlobStorageTest.java
new file mode 100644
index 0000000..1f49fd9
--- /dev/null
+++ b/asterixdb/asterix-cloud/src/test/java/org/apache/asterix/cloud/azure/LSMAzBlobStorageTest.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.cloud.azure;
+
+import org.apache.asterix.cloud.AbstractLSMTest;
+import org.apache.asterix.cloud.clients.ICloudGuardian;
+import org.apache.asterix.cloud.clients.azure.blobstorage.AzBlobStorageClientConfig;
+import org.apache.asterix.cloud.clients.azure.blobstorage.AzBlobStorageCloudClient;
+import org.apache.hyracks.util.StorageUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import com.azure.core.http.rest.PagedIterable;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.BlobServiceClientBuilder;
+import com.azure.storage.blob.models.BlobItem;
+import com.azure.storage.blob.models.ListBlobsOptions;
+import com.azure.storage.common.StorageSharedKeyCredential;
+
+public class LSMAzBlobStorageTest extends AbstractLSMTest {
+    private static BlobContainerClient client;
+
+    private static BlobServiceClient blobServiceClient;
+    private static final int MOCK_SERVER_PORT = 15055;
+    private static final String MOCK_SERVER_HOSTNAME = "http://127.0.0.1:" + MOCK_SERVER_PORT;
+    private static final String MOCK_SERVER_REGION = "us-west-2";
+
+    @BeforeClass
+    public static void setup() throws Exception {
+        LOGGER.info("LSMAzBlobStorageTest setup");
+
+        String endpointString = "http://127.0.0.1:15055/devstoreaccount1/" + PLAYGROUND_CONTAINER;
+        final String accKey =
+                "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==";
+        final String accName = "devstoreaccount1";
+
+        blobServiceClient = new BlobServiceClientBuilder().endpoint(endpointString)
+                .credential(new StorageSharedKeyCredential(accName, accKey)).buildClient();
+
+        // Start the test clean by deleting any residual data from previous tests
+        blobServiceClient.deleteBlobContainerIfExists(PLAYGROUND_CONTAINER);
+        client = blobServiceClient.createBlobContainerIfNotExists(PLAYGROUND_CONTAINER);
+
+        LOGGER.info("Az Blob Client created successfully");
+        int writeBufferSize = StorageUtil.getIntSizeInBytes(5, StorageUtil.StorageUnit.MEGABYTE);
+        AzBlobStorageClientConfig config = new AzBlobStorageClientConfig(MOCK_SERVER_REGION, MOCK_SERVER_HOSTNAME, "",
+                true, 0, PLAYGROUND_CONTAINER, 1, 0, 0, writeBufferSize);
+        CLOUD_CLIENT = new AzBlobStorageCloudClient(config, ICloudGuardian.NoOpCloudGuardian.INSTANCE);
+    }
+
+    private static void cleanup() {
+        try {
+            PagedIterable<BlobItem> blobItems = client.listBlobs(new ListBlobsOptions().setPrefix(""), null);
+            // Delete all the contents of the container
+            for (BlobItem blobItem : blobItems) {
+                BlobClient blobClient = client.getBlobClient(blobItem.getName());
+                blobClient.delete();
+            }
+            // Delete the container
+            blobServiceClient.deleteBlobContainer(PLAYGROUND_CONTAINER);
+        } catch (Exception ex) {
+            // ignore
+        }
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        LOGGER.info("Shutdown Azurite");
+        // Azure clients do not need explicit closure.
+        cleanup();
+    }
+}
diff --git a/asterixdb/asterix-cloud/src/test/java/org/apache/asterix/cloud/gcs/LSMGCSTest.java b/asterixdb/asterix-cloud/src/test/java/org/apache/asterix/cloud/gcs/LSMGCSTest.java
index ad14c54..d89c872 100644
--- a/asterixdb/asterix-cloud/src/test/java/org/apache/asterix/cloud/gcs/LSMGCSTest.java
+++ b/asterixdb/asterix-cloud/src/test/java/org/apache/asterix/cloud/gcs/LSMGCSTest.java
@@ -22,6 +22,7 @@
 import org.apache.asterix.cloud.clients.ICloudGuardian;
 import org.apache.asterix.cloud.clients.google.gcs.GCSClientConfig;
 import org.apache.asterix.cloud.clients.google.gcs.GCSCloudClient;
+import org.apache.hyracks.util.StorageUtil;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
@@ -48,7 +49,9 @@
         client.create(BucketInfo.newBuilder(PLAYGROUND_CONTAINER).setStorageClass(StorageClass.STANDARD)
                 .setLocation(MOCK_SERVER_REGION).build());
         LOGGER.info("Client created successfully");
-        GCSClientConfig config = new GCSClientConfig(MOCK_SERVER_REGION, MOCK_SERVER_HOSTNAME, "", true, 0);
+        int writeBufferSize = StorageUtil.getIntSizeInBytes(5, StorageUtil.StorageUnit.MEGABYTE);
+        GCSClientConfig config =
+                new GCSClientConfig(MOCK_SERVER_REGION, MOCK_SERVER_HOSTNAME, true, 0, writeBufferSize, "");
         CLOUD_CLIENT = new GCSCloudClient(config, ICloudGuardian.NoOpCloudGuardian.INSTANCE);
     }
 
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/AssemblerBuilderVisitor.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/AssemblerBuilderVisitor.java
index 6480c30..cb447c8 100644
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/AssemblerBuilderVisitor.java
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/AssemblerBuilderVisitor.java
@@ -18,7 +18,7 @@
  */
 package org.apache.asterix.column.assembler;
 
-import static org.apache.asterix.column.metadata.dictionary.AbstractFieldNamesDictionary.DUMMY_FIELD_NAME_INDEX;
+import static org.apache.asterix.om.dictionary.AbstractFieldNamesDictionary.DUMMY_FIELD_NAME_INDEX;
 
 import java.util.ArrayList;
 import java.util.BitSet;
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/IFieldNamesDictionary.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/IFieldNamesDictionary.java
deleted file mode 100644
index 8aa0e88..0000000
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/IFieldNamesDictionary.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.column.metadata;
-
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IValueReference;
-
-/**
- * methods for defining the fieldName dictionary
- * which is used to encode a fieldName to an index.
- */
-public interface IFieldNamesDictionary {
-    /**
-     * @return get all the inserted field names
-     */
-    List<IValueReference> getFieldNames();
-
-    /**
-     * @param fieldName fieldName byte array
-     * @return returns index if field exist, otherwise insert fieldName and return the new index
-     * @throws HyracksDataException
-     */
-    int getOrCreateFieldNameIndex(IValueReference fieldName) throws HyracksDataException;
-
-    /**
-     * @param fieldName fieldName string
-     * @return returns index if field exist, otherwise insert fieldName and return the new index
-     * @throws HyracksDataException
-     */
-    int getOrCreateFieldNameIndex(String fieldName) throws HyracksDataException;
-
-    /**
-     * @param fieldName
-     * @return index of the field if exists otherwise -1
-     * @throws HyracksDataException
-     */
-    int getFieldNameIndex(String fieldName) throws HyracksDataException;
-
-    /**
-     * @param index encoded index
-     * @return the fieldName present at the requested index
-     */
-    IValueReference getFieldName(int index);
-
-    /**
-     * serialize the dictionary
-     * @param output
-     * @throws IOException
-     */
-    void serialize(DataOutput output) throws IOException;
-
-    /**
-     * resetting and rebuilding the dictionary
-     * @param input
-     * @throws IOException
-     */
-    void abort(DataInputStream input) throws IOException;
-
-}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/dictionary/AbstractFieldNamesDictionary.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/dictionary/AbstractFieldNamesDictionary.java
deleted file mode 100644
index bffdb33..0000000
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/dictionary/AbstractFieldNamesDictionary.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.column.metadata.dictionary;
-
-import java.io.DataInput;
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.asterix.column.metadata.IFieldNamesDictionary;
-import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
-import org.apache.asterix.om.base.AMutableString;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IValueReference;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.util.string.UTF8StringReader;
-import org.apache.hyracks.util.string.UTF8StringWriter;
-
-public abstract class AbstractFieldNamesDictionary implements IFieldNamesDictionary {
-    /**
-     * Dummy field name used to add a column when encountering empty object
-     */
-    public static final IValueReference DUMMY_FIELD_NAME;
-    public static final int DUMMY_FIELD_NAME_INDEX = -1;
-
-    //For declared fields
-    private final AMutableString mutableString;
-    private final AStringSerializerDeserializer stringSerDer;
-
-    static {
-        VoidPointable dummy = new VoidPointable();
-        dummy.set(new byte[0], 0, 0);
-        DUMMY_FIELD_NAME = dummy;
-    }
-
-    AbstractFieldNamesDictionary() {
-        mutableString = new AMutableString("");
-        stringSerDer = new AStringSerializerDeserializer(new UTF8StringWriter(), new UTF8StringReader());
-    }
-
-    public static IFieldNamesDictionary create() {
-        return new FieldNamesTrieDictionary();
-    }
-
-    public static IFieldNamesDictionary deserialize(DataInput input) throws IOException {
-        return FieldNamesTrieDictionary.deserialize(input);
-    }
-
-    static ArrayBackedValueStorage creatFieldName(IValueReference fieldName) throws HyracksDataException {
-        ArrayBackedValueStorage copy = new ArrayBackedValueStorage(fieldName.getLength());
-        copy.append(fieldName);
-        return copy;
-    }
-
-    protected ArrayBackedValueStorage creatFieldName(String fieldName) throws HyracksDataException {
-        ArrayBackedValueStorage serializedFieldName = new ArrayBackedValueStorage();
-        serializeFieldName(fieldName, serializedFieldName);
-        return serializedFieldName;
-    }
-
-    protected void serializeFieldName(String fieldName, ArrayBackedValueStorage storage) throws HyracksDataException {
-        mutableString.setValue(fieldName);
-        stringSerDer.serialize(mutableString, storage.getDataOutput());
-    }
-
-    static void deserializeFieldNames(DataInput input, List<IValueReference> fieldNames, int numberOfFieldNames)
-            throws IOException {
-        for (int i = 0; i < numberOfFieldNames; i++) {
-            int length = input.readInt();
-            ArrayBackedValueStorage fieldName = new ArrayBackedValueStorage(length);
-            fieldName.setSize(length);
-            input.readFully(fieldName.getByteArray(), 0, length);
-            fieldNames.add(fieldName);
-        }
-    }
-}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/dictionary/ByteToNodeMap.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/dictionary/ByteToNodeMap.java
deleted file mode 100644
index 73c034b..0000000
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/dictionary/ByteToNodeMap.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.column.metadata.dictionary;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Collection;
-
-import it.unimi.dsi.fastutil.objects.ObjectArrays;
-
-final class ByteToNodeMap {
-    private static final TrieNode[] EMPTY = new TrieNode[0];
-    private TrieNode[] children;
-    private int numberOfChildren;
-
-    ByteToNodeMap() {
-        children = EMPTY;
-        numberOfChildren = 0;
-    }
-
-    private ByteToNodeMap(TrieNode[] children, int numberOfChildren) {
-        this.children = children;
-        this.numberOfChildren = numberOfChildren;
-    }
-
-    void put(byte key, TrieNode node) {
-        int index = Byte.toUnsignedInt(key);
-        ensure(index);
-        children[index] = node;
-        numberOfChildren++;
-    }
-
-    TrieNode get(byte key) {
-        int index = Byte.toUnsignedInt(key);
-        if (index < children.length) {
-            return children[index];
-        }
-
-        return null;
-    }
-
-    private void ensure(int index) {
-        if (index >= children.length) {
-            children = ObjectArrays.grow(children, index + 1, children.length);
-        }
-    }
-
-    void addAllChildren(Collection<TrieNode> collection) {
-        int addedChildren = 0;
-        for (int i = 0; i < children.length && addedChildren < numberOfChildren; i++) {
-            TrieNode child = children[i];
-            if (child != null) {
-                collection.add(children[i]);
-                addedChildren++;
-            }
-        }
-    }
-
-    void serialize(DataOutput out) throws IOException {
-        out.writeInt(numberOfChildren);
-        out.writeInt(children.length);
-        int addedChildren = 0;
-        for (int i = 0; i < children.length && addedChildren < numberOfChildren; i++) {
-            TrieNode child = children[i];
-            if (child != null) {
-                out.writeInt(i);
-                child.serialize(out);
-                addedChildren++;
-            }
-        }
-    }
-
-    static ByteToNodeMap deserialize(DataInput in) throws IOException {
-        int numberOfChildren = in.readInt();
-        int length = in.readInt();
-        TrieNode[] children = length == 0 ? EMPTY : new TrieNode[length];
-        for (int i = 0; i < numberOfChildren; i++) {
-            int index = in.readInt();
-            children[index] = TrieNode.deserialize(in);
-        }
-
-        return new ByteToNodeMap(children, numberOfChildren);
-    }
-}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/dictionary/FieldNameTrie.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/dictionary/FieldNameTrie.java
deleted file mode 100644
index 4a19cd6..0000000
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/dictionary/FieldNameTrie.java
+++ /dev/null
@@ -1,296 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.column.metadata.dictionary;
-
-import static org.apache.asterix.column.metadata.dictionary.AbstractFieldNamesDictionary.deserializeFieldNames;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Queue;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IValueReference;
-import org.apache.hyracks.util.string.UTF8StringUtil;
-
-public class FieldNameTrie {
-    private static final int VERSION = 1;
-    private final LookupState lookupState;
-
-    private final List<IValueReference> fieldNames;
-    private TrieNode rootNode;
-
-    public FieldNameTrie() {
-        this(new ArrayList<>());
-    }
-
-    private FieldNameTrie(List<IValueReference> fieldNames) {
-        this.fieldNames = fieldNames;
-        this.rootNode = new TrieNode();
-        lookupState = new LookupState();
-    }
-
-    public int insert(IValueReference fieldName) throws HyracksDataException {
-        int presentIndex = lookup(fieldName);
-        if (presentIndex == TrieNode.NOT_FOUND_INDEX) {
-            presentIndex = hookup(FieldNamesTrieDictionary.creatFieldName(fieldName));
-        }
-        return presentIndex;
-    }
-
-    public int lookup(IValueReference fieldName) {
-        //noinspection DuplicatedCode
-        int len = UTF8StringUtil.getUTFLength(fieldName.getByteArray(), fieldName.getStartOffset());
-        int start = fieldName.getStartOffset() + UTF8StringUtil.getNumBytesToStoreLength(len);
-        byte[] bytes = fieldName.getByteArray();
-
-        TrieNode searchNode = rootNode;
-        TrieNode prevNode = searchNode;
-
-        int byteIndex = start;
-        // previousByteIndex should point to the first byte to be compared
-        // when inserting the fieldName
-        int previousByteIndex = byteIndex;
-
-        int lastIndex = (start + len - 1);
-        while (byteIndex <= lastIndex) {
-            byte b = bytes[byteIndex];
-
-            TrieNode nextNode = searchNode.getChild(b);
-            if (nextNode == null) {
-                // saving state in case hookup is requested
-                lookupState.setState(prevNode, start, previousByteIndex, len);
-                return TrieNode.NOT_FOUND_INDEX;
-            }
-            // if the node exists, then compare the remaining byte seq.
-            prevNode = searchNode;
-            searchNode = nextNode;
-
-            if (searchNode.getLength() > 1) { // first byte will be same as byteIndex
-                // compare the stored sequence.
-                int fieldBytesLeftToCompare = lastIndex - byteIndex + 1;
-                // if the stored sequence in node is greater than the input field's
-                // byte to compare, then the result won't be there.
-                if (fieldBytesLeftToCompare < searchNode.getLength()) {
-                    // saving state in case hookup is requested
-                    lookupState.setState(prevNode, start, byteIndex, len);
-                    return TrieNode.NOT_FOUND_INDEX;
-                }
-
-                int c = 0;
-                byte[] storedFieldBytes = fieldNames.get(searchNode.getIndex()).getByteArray();
-                int storedFieldStart = searchNode.getStart();
-                previousByteIndex = byteIndex;
-                while (c < searchNode.getLength()) {
-                    if (bytes[byteIndex] != storedFieldBytes[storedFieldStart + c]) {
-                        // saving state in case hookup is requested
-                        // will restart from oldByteIndex, to make logic simpler.
-                        // other way could have been to store the splitIndex.
-                        lookupState.setState(prevNode, start, previousByteIndex, len);
-                        return TrieNode.NOT_FOUND_INDEX;
-                    }
-                    c++;
-                    byteIndex++;
-                }
-            } else {
-                previousByteIndex = byteIndex;
-                byteIndex++;
-            }
-        }
-
-        // saving state in case hookup is requested
-        lookupState.setState(prevNode, start, previousByteIndex, len);
-        return searchNode.isEndOfField() ? searchNode.getIndex() : TrieNode.NOT_FOUND_INDEX;
-    }
-
-    private int hookup(IValueReference fieldName) {
-        // since lookup operation always precedes a hookup operation
-        // we can use the saved state to start hookup.
-        int len = lookupState.getFieldLength();
-        TrieNode searchNode = lookupState.getLastNode();
-
-        // resume from the stored node.
-        int bytesToStoreLength = UTF8StringUtil.getNumBytesToStoreLength(len);
-
-        int byteIndex = lookupState.getRelativeOffsetFromStart() + bytesToStoreLength;
-        byte[] bytes = fieldName.getByteArray();
-        int lastIndex = (bytesToStoreLength + len - 1);
-        while (byteIndex <= lastIndex) {
-            byte b = bytes[byteIndex];
-            TrieNode nextNode = searchNode.getChild(b);
-            if (nextNode == null) {
-                // since there no such node, then create a node, and put
-                // rest bytes in the nodes.
-                TrieNode childNode = new TrieNode();
-                // first insert, then add the field
-                // start from byteIndex with newLength.
-                // newLength = lastIndex - byteIndex + 1
-                childNode.setIndex(fieldNames.size(), byteIndex, lastIndex - byteIndex + 1, bytesToStoreLength);
-                childNode.setIsEndOfField(true);
-                fieldNames.add(fieldName);
-
-                searchNode.putChild(b, childNode);
-                return childNode.getIndex();
-            }
-            // if node exists, compare the remaining byte seq
-            searchNode = nextNode;
-
-            if (searchNode.getLength() > 1) {
-                // compare the byte seq
-                int c = 0;
-                int oldByteIndex = byteIndex;
-
-                IValueReference storedFieldName = fieldNames.get(searchNode.getIndex());
-                byte[] storedFieldBytes = storedFieldName.getByteArray();
-                int storedFieldStart = searchNode.getStart();
-                while (c < Math.min(searchNode.getLength(), lastIndex - oldByteIndex + 1)) {
-                    if (bytes[byteIndex] != storedFieldBytes[storedFieldStart + c]) {
-                        break;
-                    }
-                    c++;
-                    byteIndex++;
-                }
-
-                // from c & byteIndex, things are not matching,
-                // split into two nodes,
-                // one from (c, ...) -> handled below
-                // other from (byteIndex, ...) -> handled in the next iteration, as byteIndex will be absent.
-
-                // handling (c, ..)
-                int leftToSplitForCurrentNode = searchNode.getLength() - c;
-                if (leftToSplitForCurrentNode > 0) {
-                    searchNode.split(storedFieldName, c);
-                }
-            } else {
-                byteIndex++;
-            }
-        }
-
-        // since the node is already present,
-        // point it to the current fieldName, and update the start and length based on the fieldName
-        // prefix would be the same
-        // find absolute starting point in the current fieldName
-        int diff = searchNode.getStart() - searchNode.getBytesToStoreLength();
-        // since hookup happens on a new fieldName, hence start will be bytesToStoreLength
-        searchNode.setIndex(fieldNames.size(), bytesToStoreLength + diff, searchNode.getLength(), bytesToStoreLength);
-        searchNode.setIsEndOfField(true);
-        fieldNames.add(fieldName);
-        return searchNode.getIndex();
-    }
-
-    public void serialize(DataOutput out) throws IOException {
-        out.writeInt(VERSION);
-
-        // serialize fieldNames
-        out.writeInt(fieldNames.size());
-        for (IValueReference fieldName : fieldNames) {
-            out.writeInt(fieldName.getLength());
-            out.write(fieldName.getByteArray(), fieldName.getStartOffset(), fieldName.getLength());
-        }
-
-        rootNode.serialize(out);
-    }
-
-    public List<IValueReference> getFieldNames() {
-        return fieldNames;
-    }
-
-    public IValueReference getFieldName(int fieldIndex) {
-        return fieldNames.get(fieldIndex);
-    }
-
-    public void clear() {
-        rootNode = null;
-        fieldNames.clear();
-    }
-
-    public static FieldNameTrie deserialize(DataInput in) throws IOException {
-        int version = in.readInt();
-        if (version == VERSION) {
-            return deserializeV1(in);
-        }
-        throw new IllegalStateException("Unsupported version: " + version);
-    }
-
-    private static FieldNameTrie deserializeV1(DataInput in) throws IOException {
-        int numberOfFieldNames = in.readInt();
-
-        List<IValueReference> fieldNames = new ArrayList<>();
-        deserializeFieldNames(in, fieldNames, numberOfFieldNames);
-
-        FieldNameTrie newTrie = new FieldNameTrie(fieldNames);
-        newTrie.rootNode = TrieNode.deserialize(in);
-
-        return newTrie;
-    }
-
-    @Override
-    public String toString() {
-        TrieNode currentNode = rootNode;
-        Queue<TrieNode> queue = new ArrayDeque<>();
-        currentNode.getChildren().addAllChildren(queue);
-        StringBuilder treeBuilder = new StringBuilder();
-        while (!queue.isEmpty()) {
-            int len = queue.size();
-            for (int i = 0; i < len; i++) {
-                TrieNode node = queue.poll();
-                assert node != null;
-                byte[] bytes = fieldNames.get(node.getIndex()).getByteArray();
-                for (int j = 0; j < node.getLength(); j++) {
-                    treeBuilder.append((char) bytes[node.getStart() + j]);
-                }
-                treeBuilder.append("(").append(node.isEndOfField()).append(")");
-                if (i != len - 1) {
-                    treeBuilder.append(" | ");
-                }
-
-                node.getChildren().addAllChildren(queue);
-            }
-            treeBuilder.append("\n");
-        }
-        return treeBuilder.toString();
-    }
-
-    private static class LookupState {
-        private TrieNode lastNode;
-        private int relativeOffsetFromStart;
-        private int fieldLength;
-
-        public void setState(TrieNode lastNode, int startIndex, int continuationByteIndex, int fieldLength) {
-            this.lastNode = lastNode;
-            this.relativeOffsetFromStart = continuationByteIndex - startIndex;
-            this.fieldLength = fieldLength;
-        }
-
-        public TrieNode getLastNode() {
-            return lastNode;
-        }
-
-        public int getRelativeOffsetFromStart() {
-            return relativeOffsetFromStart;
-        }
-
-        public int getFieldLength() {
-            return fieldLength;
-        }
-    }
-}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/dictionary/FieldNamesHashDictionary.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/dictionary/FieldNamesHashDictionary.java
deleted file mode 100644
index 73c9a73..0000000
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/dictionary/FieldNamesHashDictionary.java
+++ /dev/null
@@ -1,199 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.column.metadata.dictionary;
-
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
-import org.apache.hyracks.data.std.api.IValueReference;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-
-import it.unimi.dsi.fastutil.ints.Int2IntMap;
-import it.unimi.dsi.fastutil.ints.Int2IntOpenHashMap;
-import it.unimi.dsi.fastutil.objects.Object2IntMap;
-import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
-
-/**
- * @deprecated Use {@link FieldNamesTrieDictionary}
- */
-@Deprecated
-public class FieldNamesHashDictionary extends AbstractFieldNamesDictionary {
-    //For both declared and inferred fields
-    private final List<IValueReference> fieldNames;
-    private final Object2IntMap<String> declaredFieldNamesToIndexMap;
-    private final Int2IntMap hashToFieldNameIndexMap;
-    private final IBinaryHashFunction fieldNameHashFunction;
-
-    //For lookups
-    private final ArrayBackedValueStorage lookupStorage;
-
-    public FieldNamesHashDictionary() {
-        this(new ArrayList<>(), new Object2IntOpenHashMap<>(), new Int2IntOpenHashMap());
-    }
-
-    private FieldNamesHashDictionary(List<IValueReference> fieldNames,
-            Object2IntMap<String> declaredFieldNamesToIndexMap, Int2IntMap hashToFieldNameIndexMap) {
-        super();
-        this.fieldNames = fieldNames;
-        this.declaredFieldNamesToIndexMap = declaredFieldNamesToIndexMap;
-        this.hashToFieldNameIndexMap = hashToFieldNameIndexMap;
-        fieldNameHashFunction =
-                new PointableBinaryHashFunctionFactory(UTF8StringPointable.FACTORY).createBinaryHashFunction();
-        lookupStorage = new ArrayBackedValueStorage();
-    }
-
-    @Override
-    public List<IValueReference> getFieldNames() {
-        return fieldNames;
-    }
-
-    //TODO solve collision (they're so rare that I haven't seen any)
-    @Override
-    public int getOrCreateFieldNameIndex(IValueReference fieldName) throws HyracksDataException {
-        if (fieldName == DUMMY_FIELD_NAME) {
-            return DUMMY_FIELD_NAME_INDEX;
-        }
-
-        int hash = getHash(fieldName);
-        if (!hashToFieldNameIndexMap.containsKey(hash)) {
-            int index = addFieldName(creatFieldName(fieldName), hash);
-            hashToFieldNameIndexMap.put(hash, index);
-            return index;
-        }
-        return hashToFieldNameIndexMap.get(hash);
-    }
-
-    @Override
-    public int getOrCreateFieldNameIndex(String fieldName) throws HyracksDataException {
-        if (!declaredFieldNamesToIndexMap.containsKey(fieldName)) {
-            IValueReference serializedFieldName = creatFieldName(fieldName);
-            int hash = getHash(serializedFieldName);
-            int index = addFieldName(serializedFieldName, hash);
-            declaredFieldNamesToIndexMap.put(fieldName, index);
-            return index;
-        }
-        return declaredFieldNamesToIndexMap.getInt(fieldName);
-    }
-
-    @Override
-    public int getFieldNameIndex(String fieldName) throws HyracksDataException {
-        lookupStorage.reset();
-        serializeFieldName(fieldName, lookupStorage);
-        return hashToFieldNameIndexMap.getOrDefault(getHash(lookupStorage), -1);
-    }
-
-    private int getHash(IValueReference fieldName) throws HyracksDataException {
-        byte[] object = fieldName.getByteArray();
-        int start = fieldName.getStartOffset();
-        int length = fieldName.getLength();
-
-        return fieldNameHashFunction.hash(object, start, length);
-    }
-
-    private int addFieldName(IValueReference fieldName, int hash) {
-        int index = fieldNames.size();
-        hashToFieldNameIndexMap.put(hash, index);
-        fieldNames.add(fieldName);
-        return index;
-    }
-
-    @Override
-    public IValueReference getFieldName(int index) {
-        if (index == DUMMY_FIELD_NAME_INDEX) {
-            return DUMMY_FIELD_NAME;
-        }
-        return fieldNames.get(index);
-    }
-
-    @Override
-    public void serialize(DataOutput output) throws IOException {
-        output.writeInt(fieldNames.size());
-        for (IValueReference fieldName : fieldNames) {
-            output.writeInt(fieldName.getLength());
-            output.write(fieldName.getByteArray(), fieldName.getStartOffset(), fieldName.getLength());
-        }
-
-        output.writeInt(declaredFieldNamesToIndexMap.size());
-        for (Object2IntMap.Entry<String> declaredFieldIndex : declaredFieldNamesToIndexMap.object2IntEntrySet()) {
-            output.writeUTF(declaredFieldIndex.getKey());
-            output.writeInt(declaredFieldIndex.getIntValue());
-        }
-
-        for (Int2IntMap.Entry hashIndex : hashToFieldNameIndexMap.int2IntEntrySet()) {
-            output.writeInt(hashIndex.getIntKey());
-            output.writeInt(hashIndex.getIntValue());
-        }
-    }
-
-    public static FieldNamesHashDictionary deserialize(DataInput input) throws IOException {
-        int numberOfFieldNames = input.readInt();
-
-        List<IValueReference> fieldNames = new ArrayList<>();
-        deserializeFieldNames(input, fieldNames, numberOfFieldNames);
-
-        Object2IntMap<String> declaredFieldNamesToIndexMap = new Object2IntOpenHashMap<>();
-        deserializeDeclaredFieldNames(input, declaredFieldNamesToIndexMap);
-
-        Int2IntMap hashToFieldNameIndexMap = new Int2IntOpenHashMap();
-        deserializeHashToFieldNameIndex(input, hashToFieldNameIndexMap, numberOfFieldNames);
-
-        return new FieldNamesHashDictionary(fieldNames, declaredFieldNamesToIndexMap, hashToFieldNameIndexMap);
-    }
-
-    @Override
-    public void abort(DataInputStream input) throws IOException {
-        int numberOfFieldNames = input.readInt();
-
-        fieldNames.clear();
-        deserializeFieldNames(input, fieldNames, numberOfFieldNames);
-
-        declaredFieldNamesToIndexMap.clear();
-        deserializeDeclaredFieldNames(input, declaredFieldNamesToIndexMap);
-
-        hashToFieldNameIndexMap.clear();
-        deserializeHashToFieldNameIndex(input, hashToFieldNameIndexMap, numberOfFieldNames);
-    }
-
-    private static void deserializeDeclaredFieldNames(DataInput input,
-            Object2IntMap<String> declaredFieldNamesToIndexMap) throws IOException {
-        int numberOfDeclaredFieldNames = input.readInt();
-        for (int i = 0; i < numberOfDeclaredFieldNames; i++) {
-            String fieldName = input.readUTF();
-            int fieldNameIndex = input.readInt();
-            declaredFieldNamesToIndexMap.put(fieldName, fieldNameIndex);
-        }
-    }
-
-    private static void deserializeHashToFieldNameIndex(DataInput input, Int2IntMap hashToFieldNameIndexMap,
-            int numberOfFieldNames) throws IOException {
-        for (int i = 0; i < numberOfFieldNames; i++) {
-            int hash = input.readInt();
-            int fieldNameIndex = input.readInt();
-            hashToFieldNameIndexMap.put(hash, fieldNameIndex);
-        }
-    }
-}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/dictionary/FieldNamesTrieDictionary.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/dictionary/FieldNamesTrieDictionary.java
deleted file mode 100644
index 10de829..0000000
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/dictionary/FieldNamesTrieDictionary.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.column.metadata.dictionary;
-
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IValueReference;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-
-public class FieldNamesTrieDictionary extends AbstractFieldNamesDictionary {
-    private FieldNameTrie dictionary;
-    //For lookups
-    private final ArrayBackedValueStorage lookupStorage;
-
-    public FieldNamesTrieDictionary() {
-        this(new FieldNameTrie());
-    }
-
-    private FieldNamesTrieDictionary(FieldNameTrie dictionary) {
-        super();
-        this.dictionary = dictionary;
-        lookupStorage = new ArrayBackedValueStorage();
-    }
-
-    @Override
-    public List<IValueReference> getFieldNames() {
-        return dictionary.getFieldNames();
-    }
-
-    @Override
-    public int getOrCreateFieldNameIndex(IValueReference fieldName) throws HyracksDataException {
-        if (fieldName == DUMMY_FIELD_NAME) {
-            return DUMMY_FIELD_NAME_INDEX;
-        }
-
-        return dictionary.insert(fieldName);
-    }
-
-    @Override
-    public int getOrCreateFieldNameIndex(String fieldName) throws HyracksDataException {
-        return getOrCreateFieldNameIndex(creatFieldName(fieldName));
-    }
-
-    @Override
-    public int getFieldNameIndex(String fieldName) throws HyracksDataException {
-        lookupStorage.reset();
-        serializeFieldName(fieldName, lookupStorage);
-        return dictionary.lookup(lookupStorage);
-    }
-
-    @Override
-    public IValueReference getFieldName(int index) {
-        if (index == DUMMY_FIELD_NAME_INDEX) {
-            return DUMMY_FIELD_NAME;
-        }
-        return dictionary.getFieldName(index);
-    }
-
-    @Override
-    public void serialize(DataOutput output) throws IOException {
-        dictionary.serialize(output);
-    }
-
-    public static FieldNamesTrieDictionary deserialize(DataInput input) throws IOException {
-        FieldNameTrie fieldNameTrie = FieldNameTrie.deserialize(input);
-        return new FieldNamesTrieDictionary(fieldNameTrie);
-    }
-
-    @Override
-    public void abort(DataInputStream input) throws IOException {
-        dictionary.clear();
-        dictionary = FieldNameTrie.deserialize(input);
-    }
-}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/dictionary/TrieNode.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/dictionary/TrieNode.java
deleted file mode 100644
index 32e902b..0000000
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/dictionary/TrieNode.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.column.metadata.dictionary;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hyracks.data.std.api.IValueReference;
-
-class TrieNode {
-    public static final int NOT_FOUND_INDEX = -1;
-
-    private ByteToNodeMap children;
-    private boolean isEndOfField;
-    private int index;
-    private int start; // includes the edges' byte
-    private int length; // includes the edges' byte
-    private int bytesToStoreLength;
-
-    TrieNode() {
-        this.children = new ByteToNodeMap();
-        index = NOT_FOUND_INDEX;
-    }
-
-    TrieNode(ByteToNodeMap children) {
-        this.children = children;
-        index = NOT_FOUND_INDEX;
-    }
-
-    public void setIndex(int index, int start, int length, int bytesToStoreLength) {
-        this.index = index;
-        this.start = start;
-        this.length = length;
-        this.bytesToStoreLength = bytesToStoreLength;
-    }
-
-    public void setIsEndOfField(boolean isEndOfField) {
-        this.isEndOfField = isEndOfField;
-    }
-
-    public TrieNode getChild(byte key) {
-        return children.get(key);
-    }
-
-    public void putChild(byte key, TrieNode child) {
-        children.put(key, child);
-    }
-
-    public ByteToNodeMap getChildren() {
-        return children;
-    }
-
-    public int getIndex() {
-        return index;
-    }
-
-    public int getStart() {
-        return start;
-    }
-
-    public int getLength() {
-        return length;
-    }
-
-    public int getBytesToStoreLength() {
-        return bytesToStoreLength;
-    }
-
-    public boolean isEndOfField() {
-        return isEndOfField;
-    }
-
-    public void reset() {
-        // since this object went to the new node.
-        children = new ByteToNodeMap();
-    }
-
-    public void split(IValueReference fieldName, int splitIndex) {
-        byte[] storedFieldBytes = fieldName.getByteArray();
-        byte splitByte = storedFieldBytes[start + splitIndex];
-        // something to be split, have to create a new node
-        // and do the linking.
-        TrieNode childNode = new TrieNode(children);
-
-        int leftToSplit = length - splitIndex;
-        childNode.setIndex(index, start + splitIndex, leftToSplit, bytesToStoreLength);
-        childNode.setIsEndOfField(isEndOfField);
-        // update the current search node
-        // new length would be 'c'
-        reset();
-        setIndex(index, start, splitIndex, bytesToStoreLength);
-        putChild(splitByte, childNode);
-        // since there was a split in searchNode, hence isEndOfField will be false.
-        setIsEndOfField(false);
-    }
-
-    public void serialize(DataOutput out) throws IOException {
-        // Serialize child first
-        children.serialize(out);
-        // serialize fields
-        out.writeBoolean(isEndOfField);
-        out.writeInt(index);
-        out.writeInt(start);
-        out.writeInt(length);
-        out.writeInt(bytesToStoreLength);
-    }
-
-    public static TrieNode deserialize(DataInput in) throws IOException {
-        ByteToNodeMap children = ByteToNodeMap.deserialize(in);
-        TrieNode node = new TrieNode(children);
-        node.isEndOfField = in.readBoolean();
-        node.index = in.readInt();
-        node.start = in.readInt();
-        node.length = in.readInt();
-        node.bytesToStoreLength = in.readInt();
-
-        return node;
-    }
-}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/ObjectSchemaNode.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/ObjectSchemaNode.java
index 0bea188..1f74fb3 100644
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/ObjectSchemaNode.java
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/ObjectSchemaNode.java
@@ -18,7 +18,7 @@
  */
 package org.apache.asterix.column.metadata.schema;
 
-import static org.apache.asterix.column.metadata.dictionary.AbstractFieldNamesDictionary.DUMMY_FIELD_NAME_INDEX;
+import static org.apache.asterix.om.dictionary.AbstractFieldNamesDictionary.DUMMY_FIELD_NAME_INDEX;
 
 import java.io.DataInput;
 import java.io.DataInputStream;
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/visitor/SchemaBuilderFromIATypeVisitor.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/visitor/SchemaBuilderFromIATypeVisitor.java
index 05c4eda..c7d3df1 100644
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/visitor/SchemaBuilderFromIATypeVisitor.java
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/visitor/SchemaBuilderFromIATypeVisitor.java
@@ -22,12 +22,12 @@
 
 import java.util.List;
 
-import org.apache.asterix.column.metadata.IFieldNamesDictionary;
 import org.apache.asterix.column.metadata.schema.AbstractSchemaNode;
 import org.apache.asterix.column.metadata.schema.ObjectSchemaNode;
 import org.apache.asterix.column.metadata.schema.collection.AbstractCollectionSchemaNode;
 import org.apache.asterix.column.metadata.schema.primitive.MissingFieldSchemaNode;
 import org.apache.asterix.column.operation.lsm.flush.FlushColumnMetadata;
+import org.apache.asterix.om.dictionary.IFieldNamesDictionary;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.AbstractCollectionType;
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/visitor/SchemaClipperVisitor.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/visitor/SchemaClipperVisitor.java
index afe8368..ff05568 100644
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/visitor/SchemaClipperVisitor.java
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/metadata/schema/visitor/SchemaClipperVisitor.java
@@ -21,13 +21,13 @@
 import java.io.IOException;
 import java.util.Map;
 
-import org.apache.asterix.column.metadata.IFieldNamesDictionary;
 import org.apache.asterix.column.metadata.schema.AbstractSchemaNode;
 import org.apache.asterix.column.metadata.schema.ObjectSchemaNode;
 import org.apache.asterix.column.metadata.schema.UnionSchemaNode;
 import org.apache.asterix.column.metadata.schema.collection.AbstractCollectionSchemaNode;
 import org.apache.asterix.column.metadata.schema.primitive.MissingFieldSchemaNode;
 import org.apache.asterix.column.metadata.schema.primitive.PrimitiveSchemaNode;
+import org.apache.asterix.om.dictionary.IFieldNamesDictionary;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/FlushColumnMetadata.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/FlushColumnMetadata.java
index 337e569..5aca3a4 100644
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/FlushColumnMetadata.java
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/FlushColumnMetadata.java
@@ -33,9 +33,7 @@
 import java.util.Map;
 
 import org.apache.asterix.column.metadata.AbstractColumnMetadata;
-import org.apache.asterix.column.metadata.IFieldNamesDictionary;
 import org.apache.asterix.column.metadata.PathInfoSerializer;
-import org.apache.asterix.column.metadata.dictionary.AbstractFieldNamesDictionary;
 import org.apache.asterix.column.metadata.schema.AbstractSchemaNestedNode;
 import org.apache.asterix.column.metadata.schema.AbstractSchemaNode;
 import org.apache.asterix.column.metadata.schema.ObjectSchemaNode;
@@ -51,6 +49,8 @@
 import org.apache.asterix.column.values.IColumnValuesWriter;
 import org.apache.asterix.column.values.IColumnValuesWriterFactory;
 import org.apache.asterix.column.values.writer.AbstractColumnValuesWriter;
+import org.apache.asterix.om.dictionary.AbstractFieldNamesDictionary;
+import org.apache.asterix.om.dictionary.IFieldNamesDictionary;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.commons.lang3.mutable.Mutable;
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/NoWriteFlushColumnMetadata.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/NoWriteFlushColumnMetadata.java
index c988a11..88e6cc2 100644
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/NoWriteFlushColumnMetadata.java
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/flush/NoWriteFlushColumnMetadata.java
@@ -29,8 +29,6 @@
 import java.util.List;
 import java.util.Map;
 
-import org.apache.asterix.column.metadata.IFieldNamesDictionary;
-import org.apache.asterix.column.metadata.dictionary.AbstractFieldNamesDictionary;
 import org.apache.asterix.column.metadata.schema.AbstractSchemaNestedNode;
 import org.apache.asterix.column.metadata.schema.AbstractSchemaNode;
 import org.apache.asterix.column.metadata.schema.ObjectSchemaNode;
@@ -43,6 +41,8 @@
 import org.apache.asterix.column.values.IColumnValuesWriter;
 import org.apache.asterix.column.values.IColumnValuesWriterFactory;
 import org.apache.asterix.column.values.writer.NoOpColumnValuesWriter;
+import org.apache.asterix.om.dictionary.AbstractFieldNamesDictionary;
+import org.apache.asterix.om.dictionary.IFieldNamesDictionary;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.commons.lang3.mutable.Mutable;
@@ -199,4 +199,4 @@
     protected AbstractSchemaNode addDefinitionLevelsAndGet(AbstractSchemaNestedNode nestedNode) {
         return nestedNode;
     }
-}
\ No newline at end of file
+}
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnMetadata.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnMetadata.java
index e507d53..15a6277 100644
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnMetadata.java
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnMetadata.java
@@ -41,8 +41,6 @@
 import org.apache.asterix.column.filter.range.IColumnRangeFilterEvaluatorFactory;
 import org.apache.asterix.column.filter.range.IColumnRangeFilterValueAccessor;
 import org.apache.asterix.column.metadata.AbstractColumnImmutableReadMetadata;
-import org.apache.asterix.column.metadata.IFieldNamesDictionary;
-import org.apache.asterix.column.metadata.dictionary.AbstractFieldNamesDictionary;
 import org.apache.asterix.column.metadata.schema.AbstractSchemaNode;
 import org.apache.asterix.column.metadata.schema.ObjectSchemaNode;
 import org.apache.asterix.column.metadata.schema.visitor.SchemaClipperVisitor;
@@ -50,6 +48,8 @@
 import org.apache.asterix.column.values.IColumnValuesReader;
 import org.apache.asterix.column.values.IColumnValuesReaderFactory;
 import org.apache.asterix.column.values.reader.PrimitiveColumnValuesReader;
+import org.apache.asterix.om.dictionary.AbstractFieldNamesDictionary;
+import org.apache.asterix.om.dictionary.IFieldNamesDictionary;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.runtime.projection.FunctionCallInformation;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnWithMetaMetadata.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnWithMetaMetadata.java
index 356ddaa..d931242 100644
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnWithMetaMetadata.java
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnWithMetaMetadata.java
@@ -38,14 +38,14 @@
 import org.apache.asterix.column.filter.iterable.IColumnIterableFilterEvaluatorFactory;
 import org.apache.asterix.column.filter.range.IColumnRangeFilterEvaluatorFactory;
 import org.apache.asterix.column.filter.range.IColumnRangeFilterValueAccessor;
-import org.apache.asterix.column.metadata.IFieldNamesDictionary;
-import org.apache.asterix.column.metadata.dictionary.AbstractFieldNamesDictionary;
 import org.apache.asterix.column.metadata.schema.AbstractSchemaNode;
 import org.apache.asterix.column.metadata.schema.ObjectSchemaNode;
 import org.apache.asterix.column.metadata.schema.visitor.SchemaClipperVisitor;
 import org.apache.asterix.column.values.IColumnValuesReader;
 import org.apache.asterix.column.values.IColumnValuesReaderFactory;
 import org.apache.asterix.column.values.reader.PrimitiveColumnValuesReader;
+import org.apache.asterix.om.dictionary.AbstractFieldNamesDictionary;
+import org.apache.asterix.om.dictionary.IFieldNamesDictionary;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.runtime.projection.FunctionCallInformation;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/util/SchemaJSONBuilderVisitor.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/util/SchemaJSONBuilderVisitor.java
index b425a26..a4b9240 100644
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/util/SchemaJSONBuilderVisitor.java
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/util/SchemaJSONBuilderVisitor.java
@@ -21,7 +21,6 @@
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.asterix.column.metadata.IFieldNamesDictionary;
 import org.apache.asterix.column.metadata.schema.AbstractSchemaNode;
 import org.apache.asterix.column.metadata.schema.ISchemaNodeVisitor;
 import org.apache.asterix.column.metadata.schema.ObjectSchemaNode;
@@ -30,6 +29,7 @@
 import org.apache.asterix.column.metadata.schema.primitive.PrimitiveSchemaNode;
 import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
 import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.dictionary.IFieldNamesDictionary;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IValueReference;
 import org.apache.hyracks.data.std.util.ByteArrayAccessibleDataInputStream;
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/util/SchemaStringBuilderVisitor.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/util/SchemaStringBuilderVisitor.java
index 6d991cf..ad0e460 100644
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/util/SchemaStringBuilderVisitor.java
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/util/SchemaStringBuilderVisitor.java
@@ -21,7 +21,6 @@
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.asterix.column.metadata.IFieldNamesDictionary;
 import org.apache.asterix.column.metadata.schema.AbstractSchemaNode;
 import org.apache.asterix.column.metadata.schema.ISchemaNodeVisitor;
 import org.apache.asterix.column.metadata.schema.ObjectSchemaNode;
@@ -30,6 +29,7 @@
 import org.apache.asterix.column.metadata.schema.primitive.PrimitiveSchemaNode;
 import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
 import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.dictionary.IFieldNamesDictionary;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IValueReference;
 import org.apache.hyracks.data.std.util.ByteArrayAccessibleDataInputStream;
diff --git a/asterixdb/asterix-column/src/test/java/org/apache/asterix/column/metadata/trie/FieldNameDictionaryPerfTest.java b/asterixdb/asterix-column/src/test/java/org/apache/asterix/column/metadata/trie/FieldNameDictionaryPerfTest.java
index 63c2b22..4b33e19 100644
--- a/asterixdb/asterix-column/src/test/java/org/apache/asterix/column/metadata/trie/FieldNameDictionaryPerfTest.java
+++ b/asterixdb/asterix-column/src/test/java/org/apache/asterix/column/metadata/trie/FieldNameDictionaryPerfTest.java
@@ -20,11 +20,11 @@
 
 import java.util.concurrent.TimeUnit;
 
-import org.apache.asterix.column.metadata.IFieldNamesDictionary;
-import org.apache.asterix.column.metadata.dictionary.FieldNamesHashDictionary;
-import org.apache.asterix.column.metadata.dictionary.FieldNamesTrieDictionary;
 import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
 import org.apache.asterix.om.base.AMutableString;
+import org.apache.asterix.om.dictionary.FieldNamesHashDictionary;
+import org.apache.asterix.om.dictionary.FieldNamesTrieDictionary;
+import org.apache.asterix.om.dictionary.IFieldNamesDictionary;
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IValueReference;
diff --git a/asterixdb/asterix-column/src/test/java/org/apache/asterix/column/metadata/trie/FieldNameTrieTest.java b/asterixdb/asterix-column/src/test/java/org/apache/asterix/column/metadata/trie/FieldNameTrieTest.java
index c9f58d5..1ec468c 100644
--- a/asterixdb/asterix-column/src/test/java/org/apache/asterix/column/metadata/trie/FieldNameTrieTest.java
+++ b/asterixdb/asterix-column/src/test/java/org/apache/asterix/column/metadata/trie/FieldNameTrieTest.java
@@ -28,8 +28,8 @@
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.asterix.column.metadata.dictionary.FieldNameTrie;
-import org.apache.asterix.column.metadata.dictionary.FieldNamesTrieDictionary;
+import org.apache.asterix.om.dictionary.FieldNameTrie;
+import org.apache.asterix.om.dictionary.FieldNamesTrieDictionary;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IValueReference;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java
index eabebf7..19e4ad7 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java
@@ -29,6 +29,8 @@
 import org.apache.asterix.common.config.ReplicationProperties;
 import org.apache.asterix.common.config.StorageProperties;
 import org.apache.asterix.common.config.TransactionProperties;
+import org.apache.asterix.common.external.IExternalCredentialsCache;
+import org.apache.asterix.common.external.IExternalCredentialsCacheUpdater;
 import org.apache.hyracks.api.application.IServiceContext;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -106,4 +108,14 @@
     INamespaceResolver getNamespaceResolver();
 
     INamespacePathResolver getNamespacePathResolver();
+
+    /**
+     * @return external credentials cache
+     */
+    IExternalCredentialsCache getExternalCredentialsCache();
+
+    /**
+     * @return external credentials cache updater
+     */
+    IExternalCredentialsCacheUpdater getExternalCredentialsCacheUpdater();
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IIdentifierMapper.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IIdentifierMapper.java
index b6bce47..ba26ba7 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IIdentifierMapper.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IIdentifierMapper.java
@@ -19,13 +19,29 @@
 
 package org.apache.asterix.common.api;
 
+import org.apache.commons.lang3.StringUtils;
+
 @FunctionalInterface
 public interface IIdentifierMapper {
 
     enum Modifier {
         SINGULAR,
+        SINGULAR_CAPITALIZED,
         PLURAL,
-        NONE
+        PLURAL_CAPITALIZED,
+        NONE,
+        NONE_CAPITALIZED;
+
+        public String fixup(String input) {
+            switch (this) {
+                case SINGULAR_CAPITALIZED:
+                case PLURAL_CAPITALIZED:
+                case NONE_CAPITALIZED:
+                    return StringUtils.capitalize(input);
+                default:
+                    return input;
+            }
+        }
     }
 
     String map(String identifier, Modifier modifier);
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
index 8ca5c94..a7e39c9 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
@@ -152,7 +152,12 @@
         COMPILER_COPY_TO_WRITE_BUFFER_SIZE(
                 getRangedIntegerType(5, Integer.MAX_VALUE),
                 StorageUtil.getIntSizeInBytes(8, StorageUtil.StorageUnit.MEGABYTE),
-                "The COPY TO write buffer size in bytes. (default: 8MB, min: 5MB)");
+                "The COPY TO write buffer size in bytes. (default: 8MB, min: 5MB)"),
+        COMPILER_MAX_VARIABLE_OCCURRENCES_INLINING(
+                getRangedIntegerType(0, Integer.MAX_VALUE),
+                128,
+                "Maximum occurrences of a variable allowed in an expression for inlining"),
+        COMPILER_ORDERED_FIELDS(BOOLEAN, AlgebricksConfig.ORDERED_FIELDS, "Enable/disable select order list");
 
         private final IOptionType type;
         private final Object defaultValue;
@@ -234,6 +239,11 @@
 
     public static final String COMPILER_COLUMN_FILTER_KEY = Option.COMPILER_COLUMN_FILTER.ini();
 
+    public static final String COMPILER_MAX_VARIABLE_OCCURRENCES_INLINING_KEY =
+            Option.COMPILER_MAX_VARIABLE_OCCURRENCES_INLINING.ini();
+
+    public static final String COMPILER_ORDERED_FIELDS_KEY = Option.COMPILER_ORDERED_FIELDS.ini();
+
     public static final int COMPILER_PARALLELISM_AS_STORAGE = 0;
 
     public CompilerProperties(PropertiesAccessor accessor) {
@@ -362,6 +372,10 @@
         return accessor.getBoolean(Option.COMPILER_COLUMN_FILTER);
     }
 
+    public boolean isOrderedFields() {
+        return accessor.getBoolean(Option.COMPILER_ORDERED_FIELDS);
+    }
+
     public int getRuntimeMemoryOverheadPercentage() {
         return accessor.getInt(Option.COMPILER_RUNTIME_MEMORY_OVERHEAD);
     }
@@ -369,4 +383,8 @@
     public int getCopyToWriteBufferSize() {
         return accessor.getInt(Option.COMPILER_COPY_TO_WRITE_BUFFER_SIZE);
     }
+
+    public int getMaxVariableOccurrencesForInlining() {
+        return accessor.getInt(Option.COMPILER_MAX_VARIABLE_OCCURRENCES_INLINING);
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
index 160c04d..c866d22 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
@@ -95,6 +95,10 @@
                 compilerProperties.getQueryPlanShapeMode());
         boolean columnFilter = getBoolean(querySpecificConfig, CompilerProperties.COMPILER_COLUMN_FILTER_KEY,
                 compilerProperties.isColumnFilter());
+        int maxVariableOccurrencesForInlining =
+                getMaxVariableOccurrencesForInlining(compilerProperties, querySpecificConfig, sourceLoc);
+        boolean orderFields = getBoolean(querySpecificConfig, CompilerProperties.COMPILER_ORDERED_FIELDS_KEY,
+                compilerProperties.isOrderedFields());
 
         PhysicalOptimizationConfig physOptConf = new PhysicalOptimizationConfig();
         physOptConf.setFrameSize(frameSize);
@@ -123,6 +127,8 @@
         physOptConf.setMinJoinFrames(compilerProperties.getMinJoinMemoryFrames());
         physOptConf.setMinGroupFrames(compilerProperties.getMinGroupMemoryFrames());
         physOptConf.setMinWindowFrames(compilerProperties.getMinWindowMemoryFrames());
+        physOptConf.setMaxVariableOccurrencesForInlining(maxVariableOccurrencesForInlining);
+        physOptConf.setOrderFields(orderFields);
 
         // We should have already validated the parameter names at this point...
         Set<String> filteredParameterNames = new HashSet<>(parameterNames);
@@ -219,4 +225,16 @@
         }
         return defaultValue;
     }
+
+    private static int getMaxVariableOccurrencesForInlining(CompilerProperties compilerProperties,
+            Map<String, Object> querySpecificConfig, SourceLocation sourceLoc) throws AsterixException {
+        String valueInQuery =
+                (String) querySpecificConfig.get(CompilerProperties.COMPILER_MAX_VARIABLE_OCCURRENCES_INLINING_KEY);
+        try {
+            return valueInQuery == null ? compilerProperties.getMaxVariableOccurrencesForInlining()
+                    : OptionTypes.NONNEGATIVE_INTEGER.parse(valueInQuery);
+        } catch (IllegalArgumentException e) {
+            throw AsterixException.create(ErrorCode.COMPILATION_ERROR, sourceLoc, e.getMessage());
+        }
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index 091fb67..35e0699 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -18,6 +18,7 @@
  */
 package org.apache.asterix.common.exceptions;
 
+import org.apache.asterix.common.utils.IdentifierUtil;
 import org.apache.hyracks.api.exceptions.IError;
 import org.apache.hyracks.api.util.ErrorMessageUtil;
 
@@ -97,7 +98,6 @@
     UNSUPPORTED_COLUMN_TYPE(67),
     INVALID_KEY_TYPE(68),
     FAILED_TO_READ_KEY(69),
-
     UNSUPPORTED_JRE(100),
 
     EXTERNAL_UDF_RESULT_TYPE_ERROR(200),
@@ -304,6 +304,18 @@
     INCOMPATIBLE_FIELDS_IN_PRIMARY_KEY(1199),
     PREFIX_SHOULD_NOT_START_WITH_SLASH(1200),
     INVALID_DELTA_TABLE_FORMAT(1201),
+    UNSUPPORTED_WRITER_COMPRESSION_SCHEME(1202),
+    INVALID_PARQUET_SCHEMA(1203),
+    TYPE_UNSUPPORTED_PARQUET_WRITE(1204),
+    INVALID_PARQUET_WRITER_VERSION(1205),
+    ILLEGAL_SIZE_PROVIDED(1206),
+    TYPE_UNSUPPORTED_CSV_WRITE(1207),
+    INVALID_CSV_SCHEMA(1208),
+    MAXIMUM_VALUE_ALLOWED_FOR_PARAM(1209),
+    STORAGE_SIZE_NOT_APPLICABLE_TO_TYPE(1210),
+    COULD_NOT_CREATE_TOKENS(1211),
+    NO_AWS_VALID_PARAMS_FOUND_FOR_CROSS_ACCOUNT_TRUST_AUTHENTICATION(1212),
+    FAILED_EXTERNAL_CROSS_ACCOUNT_AUTHENTICATION(1213),
 
     // Feed errors
     DATAFLOW_ILLEGAL_STATE(3001),
@@ -423,6 +435,10 @@
     PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT(3122),
     // Avro error
     UNSUPPORTED_TYPE_FOR_AVRO(3123),
+    // Copy to CSV Error
+    INVALID_QUOTE(3124),
+    INVALID_FORCE_QUOTE(3125),
+    INVALID_ESCAPE(3126),
 
     // Lifecycle management errors
     DUPLICATE_PARTITION_ID(4000),
@@ -461,8 +477,8 @@
     }
 
     private static class ErrorMessageMapHolder {
-        private static final String[] enumMessages =
-                ErrorMessageUtil.defineMessageEnumOrdinalMap(values(), RESOURCE_PATH);
+        private static final String[] enumMessages = IdentifierUtil
+                .replaceIdentifiers(ErrorMessageUtil.defineMessageEnumOrdinalMap(values(), RESOURCE_PATH));
 
         private static String get(ErrorCode errorCode) {
             return enumMessages[errorCode.ordinal()];
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/external/IExternalCredentialsCache.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/external/IExternalCredentialsCache.java
new file mode 100644
index 0000000..245b350
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/external/IExternalCredentialsCache.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.external;
+
+import java.util.Map;
+
+public interface IExternalCredentialsCache {
+
+    /**
+     * Returns the cached credentials. Can be of any supported external credentials type
+     *
+     * @param configuration configuration containing external collection details
+     * @return credentials if present, null otherwise
+     */
+    Object getCredentials(Map<String, String> configuration);
+
+    /**
+     * Updates the credentials cache with the provided credentials for the specified name
+     *
+     * @param configuration configuration containing external collection details
+     * @param credentials credentials to cache
+     */
+    void updateCache(Map<String, String> configuration, Map<String, String> credentials);
+
+    /**
+     * Returns the name of the entity which the cached credentials belong to
+     *
+     * @param configuration configuration containing external collection details
+     * @return name of entity which credentials belong to
+     */
+    String getName(Map<String, String> configuration);
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/external/IExternalCredentialsCacheUpdater.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/external/IExternalCredentialsCacheUpdater.java
new file mode 100644
index 0000000..48553c0
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/external/IExternalCredentialsCacheUpdater.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.external;
+
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface IExternalCredentialsCacheUpdater {
+
+    /**
+     * Generates new credentials and caches them
+     *
+     * @param configuration configuration containing external collection details
+     */
+    Object generateAndCacheCredentials(Map<String, String> configuration)
+            throws HyracksDataException, CompilationException;
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/MetadataConstants.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/MetadataConstants.java
index 9b1d5ae..05b21af 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/MetadataConstants.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/MetadataConstants.java
@@ -38,6 +38,10 @@
     public static final String SYSTEM_DATABASE = "System";
     public static final String DEFAULT_DATABASE = "Default";
 
+    // Pre-defined creator
+    public static final String DEFAULT_CREATOR = "@sys";
+    public static final String DEFAULT_CREATOR_UUID = "97c793f3-bcbf-4595-8bf0-e9d6a5953523";
+
     // Name of the dataverse the metadata lives in.
     public static final DataverseName METADATA_DATAVERSE_NAME = DataverseName.createBuiltinDataverseName("Metadata");
 
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/ConstantUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/ConstantUtil.java
new file mode 100644
index 0000000..8f1ef19
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/ConstantUtil.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.common.utils;
+
+public class ConstantUtil {
+    public static final char LIKE_ESCAPE = '\\';
+    public static final char PERCENT = '%';
+    public static final char UNDERSCORE = '_';
+
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/IdentifierMappingUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/IdentifierMappingUtil.java
index 8157125..a32f7f9 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/IdentifierMappingUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/IdentifierMappingUtil.java
@@ -21,42 +21,59 @@
 
 import static org.apache.asterix.common.utils.IdentifierUtil.DATASET;
 import static org.apache.asterix.common.utils.IdentifierUtil.DATAVERSE;
+import static org.apache.asterix.common.utils.IdentifierUtil.PRODUCT_ABBREVIATION;
+import static org.apache.asterix.common.utils.IdentifierUtil.PRODUCT_NAME;
 
 import org.apache.asterix.common.api.IIdentifierMapper;
 import org.apache.asterix.common.api.IIdentifierMapper.Modifier;
 
 public class IdentifierMappingUtil {
 
+    private static final String PLAIN_DATASET = "dataset";
     private static final String SINGULAR_DATASET = "a dataset";
     private static final String PLURAL_DATASET = "datasets";
 
+    private static final String PLAIN_DATAVERSE = "dataverse";
     private static final String SINGULAR_DATAVERSE = "a dataverse";
     private static final String PLURAL_DATAVERSE = "dataverses";
 
+    private static final String DEFAULT_PRODUCT_NAME = "Apache AsterixDB";
+    private static final String DEFAULT_PRODUCT_ABBREVIATION = "AsterixDB";
+
     private static final IIdentifierMapper DEFAULT_MAPPER = (identifier, modifier) -> {
         switch (identifier) {
             case DATASET:
                 switch (modifier) {
                     case NONE:
-                        return DATASET;
+                    case NONE_CAPITALIZED:
+                        return modifier.fixup(PLAIN_DATASET);
                     case SINGULAR:
-                        return SINGULAR_DATASET;
+                    case SINGULAR_CAPITALIZED:
+                        return modifier.fixup(SINGULAR_DATASET);
                     case PLURAL:
-                        return PLURAL_DATASET;
+                    case PLURAL_CAPITALIZED:
+                        return modifier.fixup(PLURAL_DATASET);
                     default:
                         throw new IllegalArgumentException("unknown modifier " + modifier);
                 }
             case DATAVERSE:
                 switch (modifier) {
                     case NONE:
-                        return DATAVERSE;
+                    case NONE_CAPITALIZED:
+                        return modifier.fixup(PLAIN_DATAVERSE);
                     case SINGULAR:
-                        return SINGULAR_DATAVERSE;
+                    case SINGULAR_CAPITALIZED:
+                        return modifier.fixup(SINGULAR_DATAVERSE);
                     case PLURAL:
-                        return PLURAL_DATAVERSE;
+                    case PLURAL_CAPITALIZED:
+                        return modifier.fixup(PLURAL_DATAVERSE);
                     default:
                         throw new IllegalArgumentException("unknown modifier " + modifier);
                 }
+            case PRODUCT_NAME:
+                return modifier.fixup(DEFAULT_PRODUCT_NAME);
+            case PRODUCT_ABBREVIATION:
+                return modifier.fixup(DEFAULT_PRODUCT_ABBREVIATION);
             default:
                 throw new IllegalArgumentException("unmapped identifier: " + identifier);
         }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/IdentifierUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/IdentifierUtil.java
index 88b7190..addc192 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/IdentifierUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/IdentifierUtil.java
@@ -22,10 +22,22 @@
 import static org.apache.asterix.common.api.IIdentifierMapper.Modifier;
 import static org.apache.asterix.common.api.IIdentifierMapper.Modifier.NONE;
 
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.asterix.common.api.IIdentifierMapper;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
 public class IdentifierUtil {
 
-    public static final String DATASET = "dataset";
-    public static final String DATAVERSE = "dataverse";
+    private static final Logger LOGGER = LogManager.getLogger();
+    private static final Pattern MESSAGE_IDENTIFIERS = Pattern.compile("@([A-Z_]*)(:([A-Z_]*))?@");
+
+    public static final String DATASET = "DATASET";
+    public static final String DATAVERSE = "DATAVERSE";
+    public static final String PRODUCT_NAME = "PRODUCT_NAME";
+    public static final String PRODUCT_ABBREVIATION = "PRODUCT_ABBREVIATION";
 
     public static String dataset() {
         return IdentifierMappingUtil.map(DATASET, NONE);
@@ -38,4 +50,41 @@
     public static String dataverse() {
         return IdentifierMappingUtil.map(DATAVERSE, NONE);
     }
+
+    public static String productName() {
+        return IdentifierMappingUtil.map(PRODUCT_NAME, NONE);
+    }
+
+    public static String productAbbreviation() {
+        return IdentifierMappingUtil.map(PRODUCT_ABBREVIATION, NONE);
+    }
+
+    public static String replaceIdentifiers(String input) {
+        if (input == null || input.isEmpty()) {
+            return input;
+        }
+        Matcher m = MESSAGE_IDENTIFIERS.matcher(input);
+        String replacement = m.replaceAll(mr -> {
+            String identifier = mr.group(1);
+            String modifierStr = mr.group(3);
+            IIdentifierMapper.Modifier modifier;
+            if (modifierStr != null) {
+                modifier = IIdentifierMapper.Modifier.valueOf(modifierStr);
+            } else {
+                modifier = IIdentifierMapper.Modifier.NONE;
+            }
+            return IdentifierMappingUtil.map(identifier, modifier);
+        });
+        if (!input.equals(replacement)) {
+            LOGGER.debug("{} -> {}", input, replacement);
+        }
+        return replacement;
+    }
+
+    public static String[] replaceIdentifiers(String[] input) {
+        for (int i = 0; i < input.length; i++) {
+            input[i] = IdentifierUtil.replaceIdentifiers(input[i]);
+        }
+        return input;
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index 8e3ccff..d15a751 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -306,6 +306,18 @@
 1199 = Fields '%1$s' and '%2$s' are incompatible for primary key
 1200 = Prefix should not start with "/". Prefix: '%1$s'
 1201 = Supported file format for 'delta' tables is 'parquet', but '%1$s' was provided.
+1202 = Unsupported compression scheme %1$s. Supported schemes for %2$s are %3$s
+1203 = Invalid schema provided: '%1$s'
+1204 = '%1$s' type not supported in parquet format
+1205 = Invalid Parquet Writer Version provided '%1$s'. Supported values: %2$s
+1206 = Storage units expected for the field '%1$s' (e.g., 0.1KB, 100kb, 1mb, 3MB, 8.5GB ...). Provided '%2$s'
+1207 = '%1$s' type not supported in csv format
+1208 = Invalid Copy to CSV schema
+1209 = Maximum value allowed for '%1$s' is %2$s. Found %3$s
+1210 = Retrieving storage size is not applicable to type: %1$s.
+1211 = Could not create delegation tokens
+1212 = No credentials found for cross-account authentication. Expected instance profile or access key id & secret access key for assuming role
+1213 = Failed to perform cross-account authentication. Encountered error : '%1$s'
 
 # Feed Errors
 3001 = Illegal state.
@@ -427,6 +439,9 @@
 3121 = Parameter '%1$s' or '%2$s' is required if '%3$s' is provided
 3122 = Parameter '%1$s' is not allowed if '%2$s' is provided
 3123 = Type '%1$s' contains declared fields, which is not supported for 'avro' format
+3124 = '%1$s' is not a valid quote. The length of a quote should be 1
+3125 = '%1$s' is not a valid force-quote input. The length of a force-quote input should be 1 character
+3126 = '%1$s' is not a valid escape. The length of a escape should be 1
 
 # Lifecycle management errors
 4000 = Partition id %1$s for node %2$s already in use by node %3$s
diff --git a/asterixdb/asterix-doc/src/site/markdown/ncservice.md b/asterixdb/asterix-doc/src/site/markdown/ncservice.md
index c996e23..2352c02 100644
--- a/asterixdb/asterix-doc/src/site/markdown/ncservice.md
+++ b/asterixdb/asterix-doc/src/site/markdown/ncservice.md
@@ -344,6 +344,7 @@
 | common  | compiler.groupmemory                      | The memory budget (in bytes) for a group by operator instance in a partition | 33554432 (32 MB) |
 | common  | compiler.joinmemory                       | The memory budget (in bytes) for a join operator instance in a partition | 33554432 (32 MB) |
 | common  | compiler.parallelism                      | The degree of parallelism for query execution. Zero means to use the storage parallelism as the query execution parallelism, while other integer values dictate the number of query execution parallel partitions. The system will fall back to use the number of all available CPU cores in the cluster as the degree of parallelism if the number set by a user is too large or too small | 0 |
+| common  | compiler.orderfields                      | Enable/disbale select order fields in the response | false |
 | common  | compiler.sortmemory                       | The memory budget (in bytes) for a sort operator instance in a partition | 33554432 (32 MB) |
 | common  | compiler.sort.parallel                    | Enable full parallel sort for queries | true |
 | common  | compiler.sort.samples                     | The number of samples taken from each partition to guide the sort operation when full parallel sort is enabled | 100 |
diff --git a/asterixdb/asterix-external-data/pom.xml b/asterixdb/asterix-external-data/pom.xml
index 6bead12..300e723 100644
--- a/asterixdb/asterix-external-data/pom.xml
+++ b/asterixdb/asterix-external-data/pom.xml
@@ -417,8 +417,8 @@
       <artifactId>javax.activation</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.esri.geometry</groupId>
-      <artifactId>esri-geometry-api</artifactId>
+      <groupId>org.locationtech.jts</groupId>
+      <artifactId>jts-core</artifactId>
     </dependency>
     <dependency>
       <groupId>javax.xml.bind</groupId>
@@ -446,6 +446,10 @@
     </dependency>
     <dependency>
       <groupId>software.amazon.awssdk</groupId>
+      <artifactId>sts</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>software.amazon.awssdk</groupId>
       <artifactId>s3</artifactId>
     </dependency>
     <dependency>
@@ -590,6 +594,11 @@
       <artifactId>delta-standalone_2.12</artifactId>
       <version>3.0.0</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.avro</groupId>
+      <artifactId>avro-mapred</artifactId>
+      <version>1.12.0</version>
+    </dependency>
   </dependencies>
   <!-- apply patch for HADOOP-17225 to workaround CVE-2019-10172 -->
   <repositories>
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
index edd42f6..8af7342 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
@@ -22,15 +22,20 @@
 import static org.apache.asterix.external.util.ExternalDataConstants.FORMAT_PARQUET;
 import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.UUID;
 
 import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.external.IExternalFilterEvaluator;
 import org.apache.asterix.common.external.IExternalFilterEvaluatorFactory;
 import org.apache.asterix.external.api.AsterixInputStream;
 import org.apache.asterix.external.api.IExternalDataRuntimeContext;
@@ -38,19 +43,30 @@
 import org.apache.asterix.external.api.IRecordReader;
 import org.apache.asterix.external.api.IRecordReaderFactory;
 import org.apache.asterix.external.input.filter.embedder.IExternalFilterValueEmbedder;
+import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory;
 import org.apache.asterix.external.input.record.reader.hdfs.HDFSRecordReader;
+import org.apache.asterix.external.input.record.reader.hdfs.avro.AvroFileRecordReader;
 import org.apache.asterix.external.input.record.reader.hdfs.parquet.ParquetFileRecordReader;
 import org.apache.asterix.external.input.record.reader.stream.StreamRecordReader;
 import org.apache.asterix.external.input.stream.HDFSInputStream;
 import org.apache.asterix.external.provider.StreamRecordReaderProvider;
 import org.apache.asterix.external.provider.context.ExternalStreamRuntimeDataContext;
 import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataPrefix;
 import org.apache.asterix.external.util.ExternalDataUtils;
 import org.apache.asterix.external.util.HDFSUtils;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.mapred.FileInputFormat;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.application.ICCServiceContext;
@@ -58,15 +74,21 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.hyracks.api.util.ExceptionUtils;
 import org.apache.hyracks.data.std.api.IValueReference;
 import org.apache.hyracks.hdfs.dataflow.ConfFactory;
 import org.apache.hyracks.hdfs.dataflow.InputSplitsFactory;
 import org.apache.hyracks.hdfs.scheduler.Scheduler;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
 public class HDFSDataSourceFactory implements IRecordReaderFactory<Object>, IExternalDataSourceFactory {
 
     private static final long serialVersionUID = 1L;
-    private static final List<String> recordReaderNames = Collections.singletonList("hdfs");
+    private static final List<String> recordReaderNames =
+            Collections.singletonList(ExternalDataConstants.KEY_ADAPTER_NAME_HDFS);
+    private static final Logger LOGGER = LogManager.getLogger();
 
     protected transient AlgebricksAbsolutePartitionConstraint clusterLocations;
     protected transient IServiceContext serviceCtx;
@@ -85,15 +107,69 @@
     private String nodeName;
     private Class recordReaderClazz;
     private IExternalFilterEvaluatorFactory filterEvaluatorFactory;
+    private transient Credentials credentials;
+    private byte[] serializedCredentials;
+    private transient UserGroupInformation ugi;
 
     @Override
     public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
             IWarningCollector warningCollector, IExternalFilterEvaluatorFactory filterEvaluatorFactory)
             throws AlgebricksException, HyracksDataException {
         JobConf hdfsConf = prepareHDFSConf(serviceCtx, configuration, filterEvaluatorFactory);
+        credentials = HDFSUtils.configureHadoopAuthentication(configuration, hdfsConf);
+        try {
+            if (credentials != null) {
+                serializedCredentials = HDFSUtils.serialize(credentials);
+                ugi = UserGroupInformation.createRemoteUser(UUID.randomUUID().toString());
+                ugi.addCredentials(credentials);
+            }
+        } catch (IOException ex) {
+            throw HyracksDataException.create(ex);
+        }
+        if (!configuration.containsKey(ExternalDataConstants.KEY_PATH)) {
+            extractRequiredFiles(serviceCtx, configuration, warningCollector, filterEvaluatorFactory, hdfsConf);
+        }
         configureHdfsConf(hdfsConf, configuration);
     }
 
+    private void extractRequiredFiles(IServiceContext serviceCtx, Map<String, String> configuration,
+            IWarningCollector warningCollector, IExternalFilterEvaluatorFactory filterEvaluatorFactory,
+            JobConf hdfsConf) throws HyracksDataException, AlgebricksException {
+        AbstractExternalInputStreamFactory.IncludeExcludeMatcher includeExcludeMatcher =
+                ExternalDataUtils.getIncludeExcludeMatchers(configuration);
+
+        IExternalFilterEvaluator evaluator = filterEvaluatorFactory.create(serviceCtx, warningCollector);
+        ExternalDataPrefix externalDataPrefix = new ExternalDataPrefix(configuration);
+        configuration.put(ExternalDataPrefix.PREFIX_ROOT_FIELD_NAME, externalDataPrefix.getRoot());
+        try (FileSystem fs = ugi == null ? FileSystem.get(hdfsConf)
+                : ugi.doAs((PrivilegedExceptionAction<FileSystem>) () -> FileSystem.get(hdfsConf))) {
+            List<Path> reqFiles = new ArrayList<>();
+            RemoteIterator<LocatedFileStatus> files =
+                    fs.listFiles(new Path(configuration.get(ExternalDataPrefix.PREFIX_ROOT_FIELD_NAME)), true);
+            while (files.hasNext()) {
+                LocatedFileStatus file = files.next();
+                if (ExternalDataUtils.evaluate(file.getPath().toUri().getPath(), includeExcludeMatcher.getPredicate(),
+                        includeExcludeMatcher.getMatchersList(), externalDataPrefix, evaluator, warningCollector)) {
+                    reqFiles.add(file.getPath());
+                }
+            }
+            if (reqFiles.isEmpty()) {
+                if (warningCollector.shouldWarn()) {
+                    warningCollector.warn(Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES));
+                }
+                HDFSUtils.setInputDir(hdfsConf, "");
+            } else {
+                FileInputFormat.setInputPaths(hdfsConf, reqFiles.toArray(new Path[0]));
+            }
+        } catch (FileNotFoundException ex) {
+            throw CompilationException.create(ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+        } catch (InterruptedException ex) {
+            throw HyracksDataException.create(ex);
+        } catch (IOException ex) {
+            throw CompilationException.create(ErrorCode.EXTERNAL_SOURCE_ERROR, ExceptionUtils.getMessageOrToString(ex));
+        }
+    }
+
     protected JobConf prepareHDFSConf(IServiceContext serviceCtx, Map<String, String> configuration,
             IExternalFilterEvaluatorFactory filterEvaluatorFactory) throws HyracksDataException {
         this.serviceCtx = serviceCtx;
@@ -103,13 +179,15 @@
         return HDFSUtils.configureHDFSJobConf(configuration);
     }
 
-    protected void configureHdfsConf(JobConf conf, Map<String, String> configuration) throws AlgebricksException {
+    protected void configureHdfsConf(JobConf conf, Map<String, String> configuration)
+            throws AlgebricksException, HyracksDataException {
         String formatString = configuration.get(ExternalDataConstants.KEY_FORMAT);
         try {
             confFactory = new ConfFactory(conf);
             clusterLocations = getPartitionConstraint();
             int numPartitions = clusterLocations.getLocations().length;
-            InputSplit[] configInputSplits = getInputSplits(conf, numPartitions);
+            InputSplit[] configInputSplits = ugi == null ? getInputSplits(conf, numPartitions)
+                    : ugi.doAs((PrivilegedExceptionAction<InputSplit[]>) () -> getInputSplits(conf, numPartitions));
             readSchedule = hdfsScheduler.getLocationConstraints(configInputSplits);
             inputSplitsFactory = new InputSplitsFactory(configInputSplits);
             read = new boolean[readSchedule.length];
@@ -121,10 +199,14 @@
                 reader.close();
             } else if (formatString.equals(ExternalDataConstants.FORMAT_PARQUET)) {
                 recordClass = IValueReference.class;
+            } else if (formatString.equals(ExternalDataConstants.FORMAT_AVRO)) {
+                recordClass = GenericRecord.class;
             } else {
                 recordReaderClazz = StreamRecordReaderProvider.getRecordReaderClazz(configuration);
                 this.recordClass = char[].class;
             }
+        } catch (InterruptedException e) {
+            throw HyracksDataException.create(e);
         } catch (IOException e) {
             throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, e, getMessageOrToString(e));
         } catch (Exception e) {
@@ -154,10 +236,11 @@
      * 1. when target files are not null, it generates a file aware input stream that validate
      * against the files
      * 2. if the data is binary, it returns a generic reader */
-    public AsterixInputStream createInputStream(IHyracksTaskContext ctx) throws HyracksDataException {
+    public AsterixInputStream createInputStream(IHyracksTaskContext ctx, IExternalDataRuntimeContext context)
+            throws HyracksDataException {
         try {
             restoreConfig(ctx);
-            return new HDFSInputStream(read, inputSplits, readSchedule, nodeName, conf, configuration);
+            return new HDFSInputStream(read, inputSplits, readSchedule, nodeName, conf, configuration, ugi, context);
         } catch (Exception e) {
             throw HyracksDataException.create(e);
         }
@@ -169,6 +252,16 @@
             inputSplits = inputSplitsFactory.getSplits();
             nodeName = ctx.getJobletContext().getServiceContext().getNodeId();
             configured = true;
+            try {
+                if (serializedCredentials != null) {
+                    credentials = new Credentials();
+                    HDFSUtils.deserialize(serializedCredentials, credentials);
+                    ugi = UserGroupInformation.createRemoteUser(UUID.randomUUID().toString());
+                    ugi.addCredentials(credentials);
+                }
+            } catch (IOException ex) {
+                throw HyracksDataException.create(ex);
+            }
         }
     }
 
@@ -222,7 +315,7 @@
         try {
             if (recordReaderClazz != null) {
                 StreamRecordReader streamReader = (StreamRecordReader) recordReaderClazz.getConstructor().newInstance();
-                streamReader.configure(ctx, createInputStream(ctx), configuration);
+                streamReader.configure(ctx, createInputStream(ctx, context), configuration);
                 return streamReader;
             }
             restoreConfig(ctx);
@@ -237,7 +330,8 @@
                  */
                 readerConf = confFactory.getConf();
             }
-            return createRecordReader(configuration, read, inputSplits, readSchedule, nodeName, readerConf, context);
+            return createRecordReader(configuration, read, inputSplits, readSchedule, nodeName, readerConf, context,
+                    ugi);
         } catch (Exception e) {
             throw HyracksDataException.create(e);
         }
@@ -262,12 +356,15 @@
 
     private static IRecordReader<?> createRecordReader(Map<String, String> configuration, boolean[] read,
             InputSplit[] inputSplits, String[] readSchedule, String nodeName, JobConf conf,
-            IExternalDataRuntimeContext context) {
-        if (configuration.get(ExternalDataConstants.KEY_INPUT_FORMAT.trim())
+            IExternalDataRuntimeContext context, UserGroupInformation ugi) {
+        if (configuration.get(ExternalDataConstants.KEY_INPUT_FORMAT).trim()
                 .equals(ExternalDataConstants.INPUT_FORMAT_PARQUET)) {
-            return new ParquetFileRecordReader<>(read, inputSplits, readSchedule, nodeName, conf, context);
+            return new ParquetFileRecordReader<>(read, inputSplits, readSchedule, nodeName, conf, context, ugi);
+        } else if (configuration.get(ExternalDataConstants.KEY_INPUT_FORMAT).trim()
+                .equals(ExternalDataConstants.INPUT_FORMAT_AVRO)) {
+            return new AvroFileRecordReader<>(read, inputSplits, readSchedule, nodeName, conf, context, ugi);
         } else {
-            return new HDFSRecordReader<>(read, inputSplits, readSchedule, nodeName, conf);
+            return new HDFSRecordReader<>(read, inputSplits, readSchedule, nodeName, conf, ugi);
         }
     }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java
index 7a1bdad..138b364 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java
@@ -26,13 +26,14 @@
 import java.util.concurrent.TimeUnit;
 import java.util.zip.GZIPInputStream;
 
+import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.external.input.filter.embedder.IExternalFilterValueEmbedder;
 import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStream;
 import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.aws.s3.S3Utils;
+import org.apache.asterix.external.util.aws.s3.S3AuthUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.util.CleanupUtils;
@@ -48,14 +49,16 @@
 public class AwsS3InputStream extends AbstractExternalInputStream {
 
     // Configuration
+    private final IApplicationContext ncAppCtx;
     private final String bucket;
     private final S3Client s3Client;
     private ResponseInputStream<?> s3InStream;
     private static final int MAX_RETRIES = 5; // We will retry 5 times in case of internal error from AWS S3 service
 
-    public AwsS3InputStream(Map<String, String> configuration, List<String> filePaths,
+    public AwsS3InputStream(IApplicationContext ncAppCtx, Map<String, String> configuration, List<String> filePaths,
             IExternalFilterValueEmbedder valueEmbedder) throws HyracksDataException {
         super(configuration, filePaths, valueEmbedder);
+        this.ncAppCtx = ncAppCtx;
         this.s3Client = buildAwsS3Client(configuration);
         this.bucket = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
     }
@@ -113,7 +116,7 @@
     }
 
     private boolean shouldRetry(String errorCode, int currentRetry) {
-        return currentRetry < MAX_RETRIES && S3Utils.isRetryableError(errorCode);
+        return currentRetry < MAX_RETRIES && S3AuthUtils.isRetryableError(errorCode);
     }
 
     @Override
@@ -141,7 +144,7 @@
 
     private S3Client buildAwsS3Client(Map<String, String> configuration) throws HyracksDataException {
         try {
-            return S3Utils.buildAwsS3Client(configuration);
+            return S3AuthUtils.buildAwsS3Client(ncAppCtx, configuration);
         } catch (CompilationException ex) {
             throw HyracksDataException.create(ex);
         }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java
index 36d21d1..e9c72e3 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java
@@ -23,6 +23,7 @@
 import java.util.Map;
 import java.util.PriorityQueue;
 
+import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.external.IExternalFilterEvaluator;
 import org.apache.asterix.common.external.IExternalFilterEvaluatorFactory;
 import org.apache.asterix.external.api.AsterixInputStream;
@@ -47,8 +48,10 @@
     public AsterixInputStream createInputStream(IExternalDataRuntimeContext context) throws HyracksDataException {
         IExternalFilterValueEmbedder valueEmbedder = context.getValueEmbedder();
         int partition = context.getPartition();
-        return new AwsS3InputStream(configuration, partitionWorkLoadsBasedOnSize.get(partition).getFilePaths(),
-                valueEmbedder);
+        IApplicationContext ncAppCtx = (IApplicationContext) context.getTaskContext().getJobletContext()
+                .getServiceContext().getApplicationContext();
+        return new AwsS3InputStream(ncAppCtx, configuration,
+                partitionWorkLoadsBasedOnSize.get(partition).getFilePaths(), valueEmbedder);
     }
 
     @Override
@@ -65,7 +68,8 @@
         configuration.put(ExternalDataPrefix.PREFIX_ROOT_FIELD_NAME, externalDataPrefix.getRoot());
 
         // get the items
-        List<S3Object> filesOnly = S3Utils.listS3Objects(configuration, includeExcludeMatcher, warningCollector,
+        IApplicationContext appCtx = (IApplicationContext) ctx.getApplicationContext();
+        List<S3Object> filesOnly = S3Utils.listS3Objects(appCtx, configuration, includeExcludeMatcher, warningCollector,
                 externalDataPrefix, evaluator);
 
         // Distribute work load amongst the partitions
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/delta/AwsS3DeltaReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/delta/AwsS3DeltaReaderFactory.java
index d29cd40..ba0d0f4 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/delta/AwsS3DeltaReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/delta/AwsS3DeltaReaderFactory.java
@@ -18,245 +18,34 @@
  */
 package org.apache.asterix.external.input.record.reader.aws.delta;
 
-import static org.apache.asterix.external.util.aws.s3.S3Constants.SERVICE_END_POINT_FIELD_NAME;
-import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+import static org.apache.asterix.external.util.aws.s3.S3AuthUtils.configureAwsS3HdfsJobConf;
 
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.ArrayList;
 import java.util.Collections;
-import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
-import java.util.PriorityQueue;
-import java.util.Set;
 
-import org.apache.asterix.common.cluster.IClusterStateManager;
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.common.external.IExternalFilterEvaluatorFactory;
-import org.apache.asterix.external.api.IExternalDataRuntimeContext;
-import org.apache.asterix.external.api.IRecordReader;
-import org.apache.asterix.external.api.IRecordReaderFactory;
-import org.apache.asterix.external.input.record.reader.aws.delta.converter.DeltaConverterContext;
 import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.HDFSUtils;
-import org.apache.asterix.external.util.aws.s3.S3Constants;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.runtime.projection.FunctionCallInformation;
-import org.apache.commons.lang3.ArrayUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.api.application.IServiceContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.exceptions.IWarningCollector;
-import org.apache.hyracks.api.exceptions.Warning;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
+import org.apache.asterix.external.util.aws.s3.S3Utils;
+import org.apache.hadoop.mapred.JobConf;
 
-import io.delta.kernel.Scan;
-import io.delta.kernel.Snapshot;
-import io.delta.kernel.data.FilteredColumnarBatch;
-import io.delta.kernel.data.Row;
-import io.delta.kernel.defaults.engine.DefaultEngine;
-import io.delta.kernel.engine.Engine;
-import io.delta.kernel.exceptions.KernelException;
-import io.delta.kernel.internal.InternalScanFileUtils;
-import io.delta.kernel.types.StructType;
-import io.delta.kernel.utils.CloseableIterator;
-import io.delta.kernel.utils.FileStatus;
-
-public class AwsS3DeltaReaderFactory implements IRecordReaderFactory<Object> {
-
+public class AwsS3DeltaReaderFactory extends DeltaReaderFactory {
     private static final long serialVersionUID = 1L;
-    private static final List<String> recordReaderNames =
+    private static final List<String> RECORD_READER_NAMES =
             Collections.singletonList(ExternalDataConstants.KEY_ADAPTER_NAME_AWS_S3);
-    private static final Logger LOGGER = LogManager.getLogger();
-    private transient AlgebricksAbsolutePartitionConstraint locationConstraints;
-    private String scanState;
-    private Map<String, String> configuration;
-    protected final List<PartitionWorkLoadBasedOnSize> partitionWorkLoadsBasedOnSize = new ArrayList<>();
 
     @Override
-    public AlgebricksAbsolutePartitionConstraint getPartitionConstraint() {
-        return locationConstraints;
+    protected void configureJobConf(JobConf conf, Map<String, String> configuration) {
+        configureAwsS3HdfsJobConf(conf, configuration);
     }
 
     @Override
-    public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
-            IWarningCollector warningCollector, IExternalFilterEvaluatorFactory filterEvaluatorFactory)
-            throws AlgebricksException, HyracksDataException {
-        this.configuration = configuration;
-        Configuration conf = new Configuration();
-        applyConfiguration(configuration, conf);
-        String tableMetadataPath = S3Constants.HADOOP_S3_PROTOCOL + "://"
-                + configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME) + '/'
-                + configuration.get(ExternalDataConstants.DEFINITION_FIELD_NAME);
-
-        ICcApplicationContext appCtx = (ICcApplicationContext) serviceCtx.getApplicationContext();
-
-        Engine engine = DefaultEngine.create(conf);
-        io.delta.kernel.Table table = io.delta.kernel.Table.forPath(engine, tableMetadataPath);
-        Snapshot snapshot;
-        try {
-            snapshot = table.getLatestSnapshot(engine);
-        } catch (KernelException e) {
-            LOGGER.info("Failed to get latest snapshot for table: {}", tableMetadataPath, e);
-            throw RuntimeDataException.create(ErrorCode.EXTERNAL_SOURCE_ERROR, e, getMessageOrToString(e));
-        }
-
-        List<Warning> warnings = new ArrayList<>();
-        DeltaConverterContext converterContext = new DeltaConverterContext(configuration, warnings);
-        AsterixTypeToDeltaTypeVisitor visitor = new AsterixTypeToDeltaTypeVisitor(converterContext);
-        StructType requiredSchema;
-        try {
-            ARecordType expectedType = HDFSUtils.getExpectedType(conf);
-            Map<String, FunctionCallInformation> functionCallInformationMap =
-                    HDFSUtils.getFunctionCallInformationMap(conf);
-            StructType fileSchema = snapshot.getSchema(engine);
-            requiredSchema = visitor.clipType(expectedType, fileSchema, functionCallInformationMap);
-        } catch (IOException e) {
-            throw new RuntimeException(e);
-        } catch (AsterixDeltaRuntimeException e) {
-            throw e.getHyracksDataException();
-        }
-        Scan scan = snapshot.getScanBuilder(engine).withReadSchema(engine, requiredSchema).build();
-        scanState = RowSerDe.serializeRowToJson(scan.getScanState(engine));
-        CloseableIterator<FilteredColumnarBatch> iter = scan.getScanFiles(engine);
-
-        List<Row> scanFiles = new ArrayList<>();
-        while (iter.hasNext()) {
-            FilteredColumnarBatch batch = iter.next();
-            CloseableIterator<Row> rowIter = batch.getRows();
-            while (rowIter.hasNext()) {
-                Row row = rowIter.next();
-                scanFiles.add(row);
-            }
-        }
-        locationConstraints = configureLocationConstraints(appCtx, scanFiles);
-        configuration.put(ExternalDataConstants.KEY_PARSER, ExternalDataConstants.FORMAT_DELTA);
-        distributeFiles(scanFiles);
-        issueWarnings(warnings, warningCollector);
-    }
-
-    private void issueWarnings(List<Warning> warnings, IWarningCollector warningCollector) {
-        if (!warnings.isEmpty()) {
-            for (Warning warning : warnings) {
-                if (warningCollector.shouldWarn()) {
-                    warningCollector.warn(warning);
-                }
-            }
-        }
-        warnings.clear();
-    }
-
-    private AlgebricksAbsolutePartitionConstraint configureLocationConstraints(ICcApplicationContext appCtx,
-            List<Row> scanFiles) {
-        IClusterStateManager csm = appCtx.getClusterStateManager();
-
-        String[] locations = csm.getClusterLocations().getLocations();
-        if (scanFiles.size() == 0) {
-            return AlgebricksAbsolutePartitionConstraint.randomLocation(locations);
-        } else if (locations.length > scanFiles.size()) {
-            LOGGER.debug(
-                    "analytics partitions ({}) exceeds total partition count ({}); limiting ingestion partitions to total partition count",
-                    locations.length, scanFiles.size());
-            final String[] locationCopy = locations.clone();
-            ArrayUtils.shuffle(locationCopy);
-            locations = ArrayUtils.subarray(locationCopy, 0, scanFiles.size());
-        }
-        return new AlgebricksAbsolutePartitionConstraint(locations);
-    }
-
-    private void distributeFiles(List<Row> scanFiles) {
-        final int partitionsCount = getPartitionConstraint().getLocations().length;
-        PriorityQueue<PartitionWorkLoadBasedOnSize> workloadQueue = new PriorityQueue<>(partitionsCount,
-                Comparator.comparingLong(PartitionWorkLoadBasedOnSize::getTotalSize));
-
-        // Prepare the workloads based on the number of partitions
-        for (int i = 0; i < partitionsCount; i++) {
-            workloadQueue.add(new PartitionWorkLoadBasedOnSize());
-        }
-        for (Row scanFileRow : scanFiles) {
-            PartitionWorkLoadBasedOnSize workload = workloadQueue.poll();
-            FileStatus fileStatus = InternalScanFileUtils.getAddFileStatus(scanFileRow);
-            workload.addScanFile(RowSerDe.serializeRowToJson(scanFileRow), fileStatus.getSize());
-            workloadQueue.add(workload);
-        }
-        partitionWorkLoadsBasedOnSize.addAll(workloadQueue);
-    }
-
-    public static void applyConfiguration(Map<String, String> configuration, Configuration conf) {
-        conf.set(S3Constants.HADOOP_ACCESS_KEY_ID, configuration.get(S3Constants.ACCESS_KEY_ID_FIELD_NAME));
-        conf.set(S3Constants.HADOOP_SECRET_ACCESS_KEY, configuration.get(S3Constants.SECRET_ACCESS_KEY_FIELD_NAME));
-        if (configuration.get(S3Constants.SESSION_TOKEN_FIELD_NAME) != null) {
-            conf.set(S3Constants.HADOOP_SESSION_TOKEN, configuration.get(S3Constants.SESSION_TOKEN_FIELD_NAME));
-        }
-        conf.set(S3Constants.HADOOP_REGION, configuration.get(S3Constants.REGION_FIELD_NAME));
-        String serviceEndpoint = configuration.get(SERVICE_END_POINT_FIELD_NAME);
-        if (serviceEndpoint != null) {
-            conf.set(S3Constants.HADOOP_SERVICE_END_POINT, serviceEndpoint);
-        }
-        conf.set(ExternalDataConstants.KEY_REQUESTED_FIELDS,
-                configuration.getOrDefault(ExternalDataConstants.KEY_REQUESTED_FIELDS, ""));
-        conf.set(ExternalDataConstants.KEY_HADOOP_ASTERIX_FUNCTION_CALL_INFORMATION,
-                configuration.getOrDefault(ExternalDataConstants.KEY_HADOOP_ASTERIX_FUNCTION_CALL_INFORMATION, ""));
-    }
-
-    @Override
-    public IRecordReader<?> createRecordReader(IExternalDataRuntimeContext context) throws HyracksDataException {
-        try {
-            int partition = context.getPartition();
-            return new DeltaFileRecordReader(partitionWorkLoadsBasedOnSize.get(partition).getScanFiles(), scanState,
-                    configuration, context);
-        } catch (Exception e) {
-            throw HyracksDataException.create(e);
-        }
-    }
-
-    @Override
-    public Class<?> getRecordClass() throws AsterixException {
-        return Row.class;
+    protected String getTablePath(Map<String, String> configuration) {
+        return S3Utils.getPath(configuration);
     }
 
     @Override
     public List<String> getRecordReaderNames() {
-        return recordReaderNames;
-    }
-
-    @Override
-    public Set<String> getReaderSupportedFormats() {
-        return Collections.singleton(ExternalDataConstants.FORMAT_DELTA);
-    }
-
-    public static class PartitionWorkLoadBasedOnSize implements Serializable {
-        private static final long serialVersionUID = 1L;
-        private final List<String> scanFiles = new ArrayList<>();
-        private long totalSize = 0;
-
-        public PartitionWorkLoadBasedOnSize() {
-        }
-
-        public List<String> getScanFiles() {
-            return scanFiles;
-        }
-
-        public void addScanFile(String scanFile, long size) {
-            this.scanFiles.add(scanFile);
-            this.totalSize += size;
-        }
-
-        public long getTotalSize() {
-            return totalSize;
-        }
-
-        @Override
-        public String toString() {
-            return "Files: " + scanFiles.size() + ", Total Size: " + totalSize;
-        }
+        return RECORD_READER_NAMES;
     }
 
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/delta/DeltaFileRecordReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/delta/DeltaFileRecordReader.java
index a5b21b6..a094c22 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/delta/DeltaFileRecordReader.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/delta/DeltaFileRecordReader.java
@@ -19,24 +19,21 @@
 package org.apache.asterix.external.input.record.reader.aws.delta;
 
 import static io.delta.kernel.internal.util.Utils.singletonCloseableIterator;
-import static org.apache.asterix.external.util.aws.s3.S3Constants.SERVICE_END_POINT_FIELD_NAME;
 
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
 import java.util.Optional;
 
-import org.apache.asterix.external.api.IExternalDataRuntimeContext;
 import org.apache.asterix.external.api.IRawRecord;
 import org.apache.asterix.external.api.IRecordReader;
 import org.apache.asterix.external.dataflow.AbstractFeedDataFlowController;
 import org.apache.asterix.external.input.record.GenericRecord;
 import org.apache.asterix.external.util.IFeedLogManager;
-import org.apache.asterix.external.util.aws.s3.S3Constants;
-import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobConf;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.hdfs.dataflow.ConfFactory;
 
 import io.delta.kernel.Scan;
 import io.delta.kernel.data.ColumnarBatch;
@@ -69,20 +66,10 @@
     private Row scanFile;
     private CloseableIterator<Row> rows;
 
-    public DeltaFileRecordReader(List<String> serScanFiles, String serScanState, Map<String, String> conf,
-            IExternalDataRuntimeContext context) {
-        Configuration config = new Configuration();
-        config.set(S3Constants.HADOOP_ACCESS_KEY_ID, conf.get(S3Constants.ACCESS_KEY_ID_FIELD_NAME));
-        config.set(S3Constants.HADOOP_SECRET_ACCESS_KEY, conf.get(S3Constants.SECRET_ACCESS_KEY_FIELD_NAME));
-        if (conf.get(S3Constants.SESSION_TOKEN_FIELD_NAME) != null) {
-            config.set(S3Constants.HADOOP_SESSION_TOKEN, conf.get(S3Constants.SESSION_TOKEN_FIELD_NAME));
-        }
-        config.set(S3Constants.HADOOP_REGION, conf.get(S3Constants.REGION_FIELD_NAME));
-        String serviceEndpoint = conf.get(SERVICE_END_POINT_FIELD_NAME);
-        if (serviceEndpoint != null) {
-            config.set(S3Constants.HADOOP_SERVICE_END_POINT, serviceEndpoint);
-        }
-        this.engine = DefaultEngine.create(config);
+    public DeltaFileRecordReader(List<String> serScanFiles, String serScanState, ConfFactory config)
+            throws HyracksDataException {
+        JobConf conf = config.getConf();
+        this.engine = DefaultEngine.create(conf);
         this.scanFiles = new ArrayList<>();
         for (String scanFile : serScanFiles) {
             this.scanFiles.add(RowSerDe.deserializeRowFromJson(scanFile));
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/delta/DeltaReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/delta/DeltaReaderFactory.java
new file mode 100644
index 0000000..790db8c
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/delta/DeltaReaderFactory.java
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader.aws.delta;
+
+import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.Set;
+
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.common.external.IExternalFilterEvaluatorFactory;
+import org.apache.asterix.external.api.IExternalDataRuntimeContext;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.api.IRecordReaderFactory;
+import org.apache.asterix.external.input.record.reader.aws.delta.converter.DeltaConverterContext;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.HDFSUtils;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.runtime.projection.FunctionCallInformation;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.application.IServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.hyracks.hdfs.dataflow.ConfFactory;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import io.delta.kernel.Scan;
+import io.delta.kernel.Snapshot;
+import io.delta.kernel.data.FilteredColumnarBatch;
+import io.delta.kernel.data.Row;
+import io.delta.kernel.defaults.engine.DefaultEngine;
+import io.delta.kernel.engine.Engine;
+import io.delta.kernel.exceptions.KernelException;
+import io.delta.kernel.internal.InternalScanFileUtils;
+import io.delta.kernel.types.StructType;
+import io.delta.kernel.utils.CloseableIterator;
+import io.delta.kernel.utils.FileStatus;
+
+public abstract class DeltaReaderFactory implements IRecordReaderFactory<Object> {
+
+    private static final long serialVersionUID = 1L;
+    private static final Logger LOGGER = LogManager.getLogger();
+    private transient AlgebricksAbsolutePartitionConstraint locationConstraints;
+    private String scanState;
+    protected final List<PartitionWorkLoadBasedOnSize> partitionWorkLoadsBasedOnSize = new ArrayList<>();
+    protected ConfFactory confFactory;
+
+    public List<PartitionWorkLoadBasedOnSize> getPartitionWorkLoadsBasedOnSize() {
+        return partitionWorkLoadsBasedOnSize;
+    }
+
+    @Override
+    public AlgebricksAbsolutePartitionConstraint getPartitionConstraint() {
+        return locationConstraints;
+    }
+
+    protected abstract void configureJobConf(JobConf conf, Map<String, String> configuration)
+            throws AlgebricksException;
+
+    protected abstract String getTablePath(Map<String, String> configuration) throws AlgebricksException;
+
+    @Override
+    public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
+            IWarningCollector warningCollector, IExternalFilterEvaluatorFactory filterEvaluatorFactory)
+            throws AlgebricksException, HyracksDataException {
+        JobConf conf = new JobConf();
+        ICcApplicationContext appCtx = (ICcApplicationContext) serviceCtx.getApplicationContext();
+        configureJobConf(conf, configuration);
+        confFactory = new ConfFactory(conf);
+        String tableMetadataPath = getTablePath(configuration);
+        Engine engine = DefaultEngine.create(conf);
+        io.delta.kernel.Table table = io.delta.kernel.Table.forPath(engine, tableMetadataPath);
+        Snapshot snapshot;
+        try {
+            snapshot = table.getLatestSnapshot(engine);
+        } catch (KernelException e) {
+            LOGGER.info("Failed to get latest snapshot for table: {}", tableMetadataPath, e);
+            throw RuntimeDataException.create(ErrorCode.EXTERNAL_SOURCE_ERROR, e, getMessageOrToString(e));
+        }
+
+        List<Warning> warnings = new ArrayList<>();
+        DeltaConverterContext converterContext = new DeltaConverterContext(configuration, warnings);
+        AsterixTypeToDeltaTypeVisitor visitor = new AsterixTypeToDeltaTypeVisitor(converterContext);
+        StructType requiredSchema;
+        try {
+            ARecordType expectedType = HDFSUtils.getExpectedType(conf);
+            Map<String, FunctionCallInformation> functionCallInformationMap =
+                    HDFSUtils.getFunctionCallInformationMap(conf);
+            StructType fileSchema = snapshot.getSchema(engine);
+            requiredSchema = visitor.clipType(expectedType, fileSchema, functionCallInformationMap);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        } catch (AsterixDeltaRuntimeException e) {
+            throw e.getHyracksDataException();
+        }
+        Scan scan = snapshot.getScanBuilder(engine).withReadSchema(engine, requiredSchema).build();
+        scanState = RowSerDe.serializeRowToJson(scan.getScanState(engine));
+        CloseableIterator<FilteredColumnarBatch> iter = scan.getScanFiles(engine);
+
+        List<Row> scanFiles = new ArrayList<>();
+        while (iter.hasNext()) {
+            FilteredColumnarBatch batch = iter.next();
+            CloseableIterator<Row> rowIter = batch.getRows();
+            while (rowIter.hasNext()) {
+                Row row = rowIter.next();
+                scanFiles.add(row);
+            }
+        }
+        locationConstraints = getPartitions(appCtx);
+        configuration.put(ExternalDataConstants.KEY_PARSER, ExternalDataConstants.FORMAT_DELTA);
+        distributeFiles(scanFiles, getPartitionConstraint().getLocations().length);
+        issueWarnings(warnings, warningCollector);
+    }
+
+    private void issueWarnings(List<Warning> warnings, IWarningCollector warningCollector) {
+        if (!warnings.isEmpty()) {
+            for (Warning warning : warnings) {
+                if (warningCollector.shouldWarn()) {
+                    warningCollector.warn(warning);
+                }
+            }
+        }
+        warnings.clear();
+    }
+
+    public AlgebricksAbsolutePartitionConstraint getPartitions(ICcApplicationContext appCtx) {
+        return appCtx.getDataPartitioningProvider().getClusterLocations();
+    }
+
+    public void distributeFiles(List<Row> scanFiles, int partitionsCount) {
+        PriorityQueue<PartitionWorkLoadBasedOnSize> workloadQueue = new PriorityQueue<>(partitionsCount,
+                Comparator.comparingLong(PartitionWorkLoadBasedOnSize::getTotalSize));
+
+        // Prepare the workloads based on the number of partitions
+        for (int i = 0; i < partitionsCount; i++) {
+            workloadQueue.add(new PartitionWorkLoadBasedOnSize());
+        }
+        for (Row scanFileRow : scanFiles) {
+            PartitionWorkLoadBasedOnSize workload = workloadQueue.poll();
+            FileStatus fileStatus = InternalScanFileUtils.getAddFileStatus(scanFileRow);
+            workload.addScanFile(RowSerDe.serializeRowToJson(scanFileRow), fileStatus.getSize());
+            workloadQueue.add(workload);
+        }
+        partitionWorkLoadsBasedOnSize.addAll(workloadQueue);
+    }
+
+    @Override
+    public IRecordReader<?> createRecordReader(IExternalDataRuntimeContext context) throws HyracksDataException {
+        try {
+            int partition = context.getPartition();
+            return new DeltaFileRecordReader(partitionWorkLoadsBasedOnSize.get(partition).getScanFiles(), scanState,
+                    confFactory);
+        } catch (Exception e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    @Override
+    public Class<?> getRecordClass() throws AsterixException {
+        return Row.class;
+    }
+
+    @Override
+    public Set<String> getReaderSupportedFormats() {
+        return Collections.singleton(ExternalDataConstants.FORMAT_DELTA);
+    }
+
+    public static class PartitionWorkLoadBasedOnSize implements Serializable {
+        private static final long serialVersionUID = 1L;
+        private final List<String> scanFiles = new ArrayList<>();
+        private long totalSize = 0;
+
+        public PartitionWorkLoadBasedOnSize() {
+        }
+
+        public List<String> getScanFiles() {
+            return scanFiles;
+        }
+
+        public void addScanFile(String scanFile, long size) {
+            this.scanFiles.add(scanFile);
+            this.totalSize += size;
+        }
+
+        public long getTotalSize() {
+            return totalSize;
+        }
+
+        @Override
+        public String toString() {
+            return "Files: " + scanFiles.size() + ", Total Size: " + totalSize;
+        }
+    }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/parquet/AwsS3ParquetReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/parquet/AwsS3ParquetReaderFactory.java
index abed33a..7ddbab91 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/parquet/AwsS3ParquetReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/parquet/AwsS3ParquetReaderFactory.java
@@ -18,7 +18,7 @@
  */
 package org.apache.asterix.external.input.record.reader.aws.parquet;
 
-import static org.apache.asterix.external.util.aws.s3.S3Utils.configureAwsS3HdfsJobConf;
+import static org.apache.asterix.external.util.aws.s3.S3AuthUtils.configureAwsS3HdfsJobConf;
 import static org.apache.asterix.external.util.aws.s3.S3Utils.listS3Objects;
 import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
 
@@ -27,6 +27,7 @@
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.common.external.IExternalFilterEvaluator;
@@ -73,7 +74,8 @@
             configuration.put(ExternalDataPrefix.PREFIX_ROOT_FIELD_NAME, externalDataPrefix.getRoot());
 
             String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-            List<S3Object> filesOnly = listS3Objects(configuration, includeExcludeMatcher, warningCollector,
+            IApplicationContext appCtx = (IApplicationContext) serviceCtx.getApplicationContext();
+            List<S3Object> filesOnly = listS3Objects(appCtx, configuration, includeExcludeMatcher, warningCollector,
                     externalDataPrefix, evaluator);
             path = buildPathURIs(container, filesOnly);
         }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/delta/GCSDeltaReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/delta/GCSDeltaReaderFactory.java
new file mode 100644
index 0000000..ee88569
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/delta/GCSDeltaReaderFactory.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader.gcs.delta;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.external.input.record.reader.aws.delta.DeltaReaderFactory;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.google.gcs.GCSUtils;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+
+public class GCSDeltaReaderFactory extends DeltaReaderFactory {
+    private static final long serialVersionUID = 1L;
+    private static final List<String> RECORD_READER_NAMES =
+            Collections.singletonList(ExternalDataConstants.KEY_ADAPTER_NAME_GCS);
+
+    @Override
+    protected void configureJobConf(JobConf conf, Map<String, String> configuration) throws AlgebricksException {
+        GCSUtils.configureHdfsJobConf(conf, configuration);
+    }
+
+    @Override
+    protected String getTablePath(Map<String, String> configuration) throws AlgebricksException {
+        return GCSUtils.getPath(configuration);
+    }
+
+    @Override
+    public List<String> getRecordReaderNames() {
+        return RECORD_READER_NAMES;
+    }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/AbstractHDFSRecordReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/AbstractHDFSRecordReader.java
index 11241f1..352f118 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/AbstractHDFSRecordReader.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/AbstractHDFSRecordReader.java
@@ -29,6 +29,7 @@
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.security.UserGroupInformation;
 
 public abstract class AbstractHDFSRecordReader<K, V> implements IRecordReader<V> {
     protected RecordReader<K, V> reader;
@@ -43,9 +44,11 @@
     protected JobConf conf;
     protected IRawRecord<V> record;
     private boolean firstInputSplit;
+    protected UserGroupInformation ugi;
 
     public AbstractHDFSRecordReader(boolean[] read, InputSplit[] inputSplits, String[] readSchedule, String nodeName,
-            JobConf conf) {
+            JobConf conf, UserGroupInformation ugi) {
+        this.ugi = ugi;
         this.read = read;
         this.inputSplits = inputSplits;
         this.readSchedule = readSchedule;
@@ -58,7 +61,8 @@
     }
 
     public AbstractHDFSRecordReader(boolean[] read, InputSplit[] inputSplits, String[] readSchedule, String nodeName,
-            IRawRecord<V> record, JobConf conf) {
+            IRawRecord<V> record, JobConf conf, UserGroupInformation ugi) {
+        this.ugi = ugi;
         this.read = read;
         this.inputSplits = inputSplits;
         this.readSchedule = readSchedule;
@@ -82,14 +86,11 @@
             nextInputSplit();
         }
 
-        if (reader.next(key, value)) {
-            return true;
-        }
-        while (nextInputSplit()) {
-            if (reader.next(key, value)) {
+        do {
+            if (readerHasNext()) {
                 return true;
             }
-        }
+        } while (nextInputSplit());
         return false;
     }
 
@@ -99,6 +100,10 @@
         return record;
     }
 
+    protected boolean readerHasNext() throws IOException {
+        return reader.next(key, value);
+    }
+
     private boolean nextInputSplit() throws IOException {
         for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
             /**
@@ -124,14 +129,18 @@
                     continue;
                 }
 
-                reader.close();
-                reader = getRecordReader(currentSplitIndex);
+                closeRecordReader();
+                setRecordReader(currentSplitIndex);
                 return true;
             }
         }
         return false;
     }
 
+    protected void closeRecordReader() throws IOException {
+        reader.close();
+    }
+
     /**
      * Returns true if need to go to next split without closing the current reader
      *
@@ -139,7 +148,7 @@
      */
     protected abstract boolean onNextInputSplit() throws IOException;
 
-    protected abstract RecordReader<K, V> getRecordReader(int splitIndex) throws IOException;
+    protected abstract void setRecordReader(int splitIndex) throws IOException;
 
     @Override
     public boolean stop() {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSRecordReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSRecordReader.java
index 48b99d1..f143188 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSRecordReader.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSRecordReader.java
@@ -19,29 +19,40 @@
 package org.apache.asterix.external.input.record.reader.hdfs;
 
 import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
 
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class HDFSRecordReader<K, V extends Writable> extends AbstractHDFSRecordReader<K, V> {
 
     public HDFSRecordReader(boolean[] read, InputSplit[] inputSplits, String[] readSchedule, String nodeName,
-            JobConf conf) {
-        super(read, inputSplits, readSchedule, nodeName, conf);
+            JobConf conf, UserGroupInformation ugi) {
+        super(read, inputSplits, readSchedule, nodeName, conf, ugi);
     }
 
-    @SuppressWarnings("unchecked")
     @Override
-    protected RecordReader<K, V> getRecordReader(int splitIndex) throws IOException {
-        reader = (RecordReader<K, V>) inputFormat.getRecordReader(inputSplits[splitIndex], conf, Reporter.NULL);
+    protected void setRecordReader(int splitIndex) throws IOException {
+        try {
+            reader = ugi == null ? getReader(splitIndex)
+                    : ugi.doAs((PrivilegedExceptionAction<RecordReader<K, V>>) () -> getReader(splitIndex));
+        } catch (InterruptedException ex) {
+            throw HyracksDataException.create(ex);
+        }
         if (key == null) {
             key = reader.createKey();
             value = reader.createValue();
         }
-        return reader;
+    }
+
+    @SuppressWarnings("unchecked")
+    private RecordReader<K, V> getReader(int splitIndex) throws IOException {
+        return (RecordReader<K, V>) inputFormat.getRecordReader(inputSplits[splitIndex], conf, Reporter.NULL);
     }
 
     @Override
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/avro/AvroFileInputFormat.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/avro/AvroFileInputFormat.java
new file mode 100644
index 0000000..3c279a4
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/avro/AvroFileInputFormat.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader.hdfs.avro;
+
+import org.apache.avro.mapred.AvroInputFormat;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.io.compress.SplittableCompressionCodec;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobConfigurable;
+
+public class AvroFileInputFormat<T> extends AvroInputFormat<T> implements JobConfigurable {
+    private CompressionCodecFactory compressionCodecs = null;
+
+    public void configure(JobConf conf) {
+        compressionCodecs = new CompressionCodecFactory(conf);
+    }
+
+    @Override
+    public boolean isSplitable(FileSystem fs, Path file) {
+        final CompressionCodec codec = compressionCodecs.getCodec(file);
+        if (null == codec) {
+            return true;
+        }
+        return codec instanceof SplittableCompressionCodec;
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/avro/AvroFileRecordReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/avro/AvroFileRecordReader.java
new file mode 100644
index 0000000..e505666
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/avro/AvroFileRecordReader.java
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader.hdfs.avro;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.PrivilegedExceptionAction;
+import java.util.zip.GZIPInputStream;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.external.api.IExternalDataRuntimeContext;
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.input.filter.embedder.IExternalFilterValueEmbedder;
+import org.apache.asterix.external.input.record.GenericRecord;
+import org.apache.asterix.external.input.record.reader.hdfs.AbstractHDFSRecordReader;
+import org.apache.asterix.external.input.record.reader.hdfs.EmptyRecordReader;
+import org.apache.avro.InvalidAvroMagicException;
+import org.apache.avro.file.DataFileStream;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.mapred.AvroWrapper;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.ExceptionUtils;
+
+public class AvroFileRecordReader<T> extends AbstractHDFSRecordReader<Void, T> {
+
+    private AvroWrapper<T> key;
+    private NullWritable value;
+    private final IRawRecord<T> record;
+    private RecordReader<AvroWrapper<T>, NullWritable> reader;
+    private final IExternalFilterValueEmbedder valueEmbedder;
+    private boolean isCompressed = false;
+    private DataFileStream<T> dataFileStream;
+    private FileSystem fs;
+    private InputStream in;
+
+    public AvroFileRecordReader(boolean[] read, InputSplit[] inputSplits, String[] readSchedule, String nodeName,
+            JobConf conf, IExternalDataRuntimeContext context, UserGroupInformation ugi) {
+        super(read, inputSplits, readSchedule, nodeName, conf, ugi);
+        reader = new EmptyRecordReader<>();
+        record = new GenericRecord<>();
+        valueEmbedder = context.getValueEmbedder();
+    }
+
+    @Override
+    protected boolean onNextInputSplit() {
+        return false;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected void setRecordReader(int splitIndex) throws IOException {
+        try {
+            String filePath = getPath(inputSplits[splitIndex]);
+            valueEmbedder.setPath(filePath);
+            if (StringUtils.endsWithIgnoreCase(filePath, ".gz") || StringUtils.endsWithIgnoreCase(filePath, ".gzip")) {
+                isCompressed = true;
+                fs = ugi == null ? FileSystem.get(conf)
+                        : ugi.doAs((PrivilegedExceptionAction<FileSystem>) () -> FileSystem.get(conf));
+                in = new GZIPInputStream(fs.open(new Path(filePath)));
+                GenericDatumReader<T> datumReader = new GenericDatumReader<>();
+                dataFileStream = new DataFileStream<>(in, datumReader);
+            } else {
+                isCompressed = false;
+                reader = (RecordReader<AvroWrapper<T>, NullWritable>) (ugi == null
+                        ? inputFormat.getRecordReader(inputSplits[splitIndex], conf, Reporter.NULL)
+                        : ugi.doAs((PrivilegedExceptionAction<?>) () -> inputFormat
+                                .getRecordReader(inputSplits[splitIndex], conf, Reporter.NULL)));
+                if (key == null) {
+                    key = reader.createKey();
+                    value = reader.createValue();
+                }
+            }
+        } catch (InterruptedException ex) {
+            throw HyracksDataException.create(ex);
+        } catch (InvalidAvroMagicException ex) {
+            throw RuntimeDataException.create(ErrorCode.EXTERNAL_SOURCE_ERROR, ex,
+                    ExceptionUtils.getMessageOrToString(ex));
+        }
+    }
+
+    @Override
+    protected void closeRecordReader() throws IOException {
+        if (isCompressed) {
+            dataFileStream.close();
+            in.close();
+            fs.close();
+        } else {
+            reader.close();
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        if (isCompressed) {
+            dataFileStream.close();
+            in.close();
+            fs.close();
+        } else {
+            reader.close();
+        }
+    }
+
+    @Override
+    protected boolean readerHasNext() throws IOException {
+        if (isCompressed) {
+            return dataFileStream.hasNext();
+        } else {
+            return reader.next(key, value);
+        }
+    }
+
+    @Override
+    public IRawRecord<T> next() throws IOException {
+        if (isCompressed) {
+            record.set(dataFileStream.next());
+        } else {
+            record.set(key.datum());
+        }
+        return record;
+    }
+
+    private String getPath(InputSplit split) {
+        if (split instanceof FileSplit) {
+            return ((FileSplit) split).getPath().toString();
+        } else {
+            return split.toString();
+        }
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ParquetFileRecordReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ParquetFileRecordReader.java
index bb74cd6..1d123b6 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ParquetFileRecordReader.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ParquetFileRecordReader.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.external.input.record.reader.hdfs.parquet;
 
 import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
 
 import org.apache.asterix.external.api.IExternalDataRuntimeContext;
 import org.apache.asterix.external.input.record.ValueReferenceRecord;
@@ -28,6 +29,8 @@
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
 import org.apache.hyracks.data.std.api.IValueReference;
 
@@ -39,8 +42,8 @@
     private final IWarningCollector warningCollector;
 
     public ParquetFileRecordReader(boolean[] read, InputSplit[] inputSplits, String[] readSchedule, String nodeName,
-            JobConf conf, IExternalDataRuntimeContext context) {
-        super(read, inputSplits, readSchedule, nodeName, new ValueReferenceRecord<>(), conf);
+            JobConf conf, IExternalDataRuntimeContext context, UserGroupInformation ugi) {
+        super(read, inputSplits, readSchedule, nodeName, new ValueReferenceRecord<>(), conf, ugi);
         this.warningCollector = context.getTaskContext().getWarningCollector();
         ((MapredParquetInputFormat) inputFormat).setValueEmbedder(context.getValueEmbedder());
     }
@@ -59,17 +62,22 @@
 
     @SuppressWarnings("unchecked")
     @Override
-    protected RecordReader<Void, V> getRecordReader(int splitIndex) throws IOException {
+    protected void setRecordReader(int splitIndex) throws IOException {
         try {
-            ParquetRecordReaderWrapper readerWrapper = (ParquetRecordReaderWrapper) inputFormat
-                    .getRecordReader(inputSplits[splitIndex], conf, Reporter.NULL);
+            ParquetRecordReaderWrapper readerWrapper = ugi == null ? getReader(splitIndex)
+                    : ugi.doAs((PrivilegedExceptionAction<ParquetRecordReaderWrapper>) () -> getReader(splitIndex));
             reader = (RecordReader<Void, V>) readerWrapper;
         } catch (AsterixParquetRuntimeException e) {
             throw e.getHyracksDataException();
+        } catch (InterruptedException e) {
+            throw HyracksDataException.create(e);
         }
         if (value == null) {
             value = reader.createValue();
         }
-        return reader;
+    }
+
+    private ParquetRecordReaderWrapper getReader(int splitIndex) throws IOException {
+        return (ParquetRecordReaderWrapper) inputFormat.getRecordReader(inputSplits[splitIndex], conf, Reporter.NULL);
     }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/HDFSInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/HDFSInputStream.java
index 46c2102..c1bad6c 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/HDFSInputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/HDFSInputStream.java
@@ -19,18 +19,24 @@
 package org.apache.asterix.external.input.stream;
 
 import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
 import java.util.Map;
 
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.external.api.AsterixInputStream;
+import org.apache.asterix.external.api.IExternalDataRuntimeContext;
+import org.apache.asterix.external.input.filter.embedder.IExternalFilterValueEmbedder;
 import org.apache.asterix.external.input.record.reader.hdfs.EmptyRecordReader;
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileSplit;
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class HDFSInputStream extends AsterixInputStream {
 
@@ -45,10 +51,15 @@
     private String nodeName;
     private JobConf conf;
     private int pos = 0;
+    private UserGroupInformation ugi;
+    private IExternalFilterValueEmbedder valueEmbedder;
 
     @SuppressWarnings("unchecked")
     public HDFSInputStream(boolean read[], InputSplit[] inputSplits, String[] readSchedule, String nodeName,
-            JobConf conf, Map<String, String> configuration) throws IOException, AsterixException {
+            JobConf conf, Map<String, String> configuration, UserGroupInformation ugi,
+            IExternalDataRuntimeContext context) throws IOException, AsterixException {
+        this.ugi = ugi;
+        this.valueEmbedder = context.getValueEmbedder();
         this.read = read;
         this.inputSplits = inputSplits;
         this.readSchedule = readSchedule;
@@ -162,13 +173,32 @@
         return false;
     }
 
-    @SuppressWarnings("unchecked")
     private RecordReader<Object, Text> getRecordReader(int splitIndex) throws IOException {
-        reader = (RecordReader<Object, Text>) inputFormat.getRecordReader(inputSplits[splitIndex], conf, Reporter.NULL);
+        valueEmbedder.setPath(getPath(inputSplits[splitIndex]));
+        try {
+            reader = ugi == null ? getReader(splitIndex)
+                    : ugi.doAs((PrivilegedExceptionAction<RecordReader<Object, Text>>) () -> getReader(splitIndex));
+        } catch (InterruptedException ex) {
+            throw HyracksDataException.create(ex);
+        }
+
         if (key == null) {
             key = reader.createKey();
             value = reader.createValue();
         }
         return reader;
     }
+
+    @SuppressWarnings("unchecked")
+    private RecordReader<Object, Text> getReader(int splitIndex) throws IOException {
+        return (RecordReader<Object, Text>) inputFormat.getRecordReader(inputSplits[splitIndex], conf, Reporter.NULL);
+    }
+
+    private String getPath(InputSplit split) {
+        if (split instanceof FileSplit) {
+            return ((FileSplit) split).getPath().toString();
+        } else {
+            return split.toString();
+        }
+    }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
index ffe75cb..46a1b5b 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
@@ -18,11 +18,13 @@
  */
 package org.apache.asterix.external.util;
 
+import java.util.List;
 import java.util.Set;
 import java.util.function.LongSupplier;
 import java.util.function.Supplier;
 import java.util.regex.Pattern;
 
+import org.apache.asterix.external.input.record.reader.hdfs.avro.AvroFileInputFormat;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.hyracks.util.StorageUtil;
 
@@ -38,6 +40,7 @@
     // used to specify the stream factory for an adapter that has a stream data source
     public static final String KEY_STREAM = "stream";
     //TODO(DB): check adapter configuration
+    public static final String KEY_DATASET = "dataset";
     public static final String KEY_DATASET_DATABASE = "dataset-database";
     // used to specify the dataverse of the adapter
     public static final String KEY_DATASET_DATAVERSE = "dataset-dataverse";
@@ -81,9 +84,17 @@
     public static final String KEY_EXPRESSION = "expression";
     public static final String KEY_LOCAL_SOCKET_PATH = "local-socket-path";
     public static final String KEY_FORMAT = "format";
+    public static final String KEY_SCHEMA = "schema";
+    public static final String KEY_PARQUET_ROW_GROUP_SIZE = "row-group-size";
+    public static final String PARQUET_DEFAULT_ROW_GROUP_SIZE = "10MB";
+    public static final String KEY_PARQUET_PAGE_SIZE = "page-size";
+    public static final String PARQUET_DEFAULT_PAGE_SIZE = "8KB";
     public static final String KEY_INCLUDE = "include";
     public static final String KEY_EXCLUDE = "exclude";
     public static final String KEY_QUOTE = "quote";
+    public static final String KEY_FORCE_QUOTE = "force-quote";
+    public static final String KEY_EMPTY_FIELD_AS_NULL = "empty_field_as_null";
+    public static final String KEY_RECORD_DELIMITER = "record-delimiter";
     public static final String KEY_ESCAPE = "escape";
     public static final String KEY_PARSER = "parser";
     public static final String KEY_DATASET_RECORD = "dataset-record";
@@ -151,6 +162,13 @@
     public static final String KEY_ADAPTER_NAME_AZURE_BLOB = "AZUREBLOB";
     public static final String KEY_ADAPTER_NAME_AZURE_DATA_LAKE = "AZUREDATALAKE";
     public static final String KEY_ADAPTER_NAME_GCS = "GCS";
+    public static final String KEY_ADAPTER_NAME_HDFS = "HDFS";
+
+    public static final Set<String> EXTERNAL_READ_ADAPTERS = Set.of(KEY_ADAPTER_NAME_TWITTER_PUSH,
+            KEY_ADAPTER_NAME_PUSH_TWITTER, KEY_ADAPTER_NAME_TWITTER_PULL, KEY_ADAPTER_NAME_PULL_TWITTER,
+            KEY_ADAPTER_NAME_TWITTER_USER_STREAM, KEY_ADAPTER_NAME_LOCALFS, KEY_ADAPTER_NAME_SOCKET,
+            KEY_ADAPTER_NAME_HTTP, KEY_ADAPTER_NAME_AWS_S3, KEY_ADAPTER_NAME_AZURE_BLOB,
+            KEY_ADAPTER_NAME_AZURE_DATA_LAKE, KEY_ADAPTER_NAME_GCS, KEY_ADAPTER_NAME_HDFS);
 
     /**
      * HDFS class names
@@ -159,13 +177,44 @@
     public static final String CLASS_NAME_SEQUENCE_INPUT_FORMAT = "org.apache.hadoop.mapred.SequenceFileInputFormat";
     public static final String CLASS_NAME_PARQUET_INPUT_FORMAT =
             "org.apache.asterix.external.input.record.reader.hdfs.parquet.MapredParquetInputFormat";
+    public static final String CLASS_NAME_AVRO_INPUT_FORMAT = AvroFileInputFormat.class.getName();
     public static final String CLASS_NAME_HDFS_FILESYSTEM = "org.apache.hadoop.hdfs.DistributedFileSystem";
+    public static final String S3A_CHANGE_DETECTION_REQUIRED = "requireVersionChangeDetection";
+    public static final String S3A_CHANGE_DETECTION_REQUIRED_CONFIG_KEY = "fs.s3a.change.detection.version.required";
+    public static final String HDFS_IO_COMPRESSION_CODECS_KEY = "io.compression.codecs";
+    public static final String HDFS_AVRO_IGNORE_INPUTS_WITHOUT_EXTENSION =
+            "avro.mapred.ignore.inputs.without.extension";
+
     /**
      * input formats aliases
      */
     public static final String INPUT_FORMAT_TEXT = "text-input-format";
     public static final String INPUT_FORMAT_SEQUENCE = "sequence-input-format";
     public static final String INPUT_FORMAT_PARQUET = "parquet-input-format";
+    public static final String INPUT_FORMAT_AVRO = "avro-input-format";
+
+    public static final String HDFS_BLOCKSIZE = "blocksize";
+    public static final String HDFS_REPLICATION = "replication";
+    public static final String HADOOP_AUTHENTICATION = "authentication";
+    public static final String KERBEROS_PROTOCOL = "kerberos";
+    public static final String KERBEROS_REALM = "realm";
+    public static final String KERBEROS_KDC = "kdc";
+    public static final String HDFS_USE_DATANODE_HOSTNAME = "use-datanode-hostname";
+    public static final String KERBEROS_PRINCIPAL = "principal";
+    public static final String KERBEROS_PASSWORD = "password";
+
+    public static final String KEY_HDFS_BLOCKSIZE = "dfs.blocksize";
+    public static final String KEY_HDFS_REPLICATION = "dfs.replication";
+    public static final String KEY_HADOOP_AUTHENTICATION = "hadoop.security.authentication";
+    public static final String KEY_KERBEROS_CONF = "java.security.krb5.conf";
+    public static final String KEY_NAMENODE_PRINCIPAL_PATTERN = "dfs.namenode.kerberos.principal.pattern";
+    public static final String KEY_HDFS_USE_DATANODE_HOSTNAME = "dfs.client.use.datanode.hostname";
+    public static final String KERBEROS_LOGIN_MODULE = "com.sun.security.auth.module.Krb5LoginModule";
+    public static final String KERBEROS_CONFIG_REFRESH = "refreshKrb5Config";
+    public static final String KERBEROS_CONFIG_FILE_CONTENT =
+            "[libdefaults]\n\tdefault_realm = %1$s\n\n[realms]\n\t%1$s = {\n\t\tkdc = %2$s\n\t}";
+    public static final String[] KERBEROS_CONFIG_FILE_PATTERN = { "krb5", ".conf" };
+
     /**
      * Builtin streams
      */
@@ -180,6 +229,8 @@
     public static final String HAS_HEADER = "has.header";
     public static final String TIME_TRACKING = "time.tracking";
     public static final String DEFAULT_QUOTE = "\"";
+    public static final String DEFAULT_SINGLE_QUOTE = "'";
+    public static final String NONE = "none";
     public static final String NODE_RESOLVER_FACTORY_PROPERTY = "node.Resolver";
     public static final String DEFAULT_DELIMITER = ",";
     public static final String EXTERNAL_LIBRARY_SEPARATOR = "#";
@@ -191,6 +242,7 @@
     public static final String FORMAT_ADM = "adm";
     public static final String FORMAT_AVRO = "avro";
     public static final String FORMAT_JSON_LOWER_CASE = "json";
+    public static final String FORMAT_CSV_LOWER_CASE = "csv";
     public static final String FORMAT_JSON_UPPER_CASE = "JSON";
     public static final String FORMAT_DELIMITED_TEXT = "delimited-text";
     public static final String FORMAT_TWEET = "twitter-status";
@@ -203,6 +255,13 @@
     public static final String FORMAT_CSV = "csv";
     public static final String FORMAT_TSV = "tsv";
     public static final String FORMAT_PARQUET = "parquet";
+    public static final String PARQUET_SCHEMA_KEY = "parquet-schema";
+    public static final String PARQUET_WRITER_VERSION_KEY = "version";
+    public static final String PARQUET_WRITER_VERSION_VALUE_1 = "1";
+    public static final String PARQUET_WRITER_VERSION_VALUE_2 = "2";
+    public static final String DUMMY_DATABASE_NAME = "dbname";
+    public static final String DUMMY_TYPE_NAME = "typeName";
+    public static final String DUMMY_DATAVERSE_NAME = "a.b.c";
     public static final String FORMAT_APACHE_ICEBERG = "apache-iceberg";
     public static final String FORMAT_DELTA = "delta";
     public static final Set<String> ALL_FORMATS;
@@ -306,6 +365,8 @@
      * Compression constants
      */
     public static final String KEY_COMPRESSION_GZIP = "gzip";
+    public static final String KEY_COMPRESSION_SNAPPY = "snappy";
+    public static final String KEY_COMPRESSION_ZSTD = "zstd";
     public static final String KEY_COMPRESSION_GZIP_COMPRESSION_LEVEL = "gzipCompressionLevel";
 
     /**
@@ -318,13 +379,28 @@
     public static final int WRITER_MAX_RESULT_MINIMUM = 1000;
     public static final Set<String> WRITER_SUPPORTED_FORMATS;
     public static final Set<String> WRITER_SUPPORTED_ADAPTERS;
-    public static final Set<String> WRITER_SUPPORTED_COMPRESSION;
+    public static final Set<String> TEXTUAL_WRITER_SUPPORTED_COMPRESSION;
+    public static final Set<String> PARQUET_WRITER_SUPPORTED_COMPRESSION;
+    public static final Set<String> PARQUET_WRITER_SUPPORTED_VERSION;
+    public static final int PARQUET_DICTIONARY_PAGE_SIZE = 1048576;
+    public static final List<String> WRITER_SUPPORTED_QUOTES;
+    public static final List<ATypeTag> CSV_WRITER_SUPPORTED_DATA_TYPES =
+            List.of(ATypeTag.TINYINT, ATypeTag.SMALLINT, ATypeTag.INTEGER, ATypeTag.BIGINT, ATypeTag.UINT8,
+                    ATypeTag.UINT16, ATypeTag.UINT64, ATypeTag.FLOAT, ATypeTag.DOUBLE, ATypeTag.STRING,
+                    ATypeTag.BOOLEAN, ATypeTag.DATETIME, ATypeTag.UINT32, ATypeTag.DATE, ATypeTag.TIME);
+    public static final String PARQUET_MAX_SCHEMAS_KEY = "max-schemas";
+    public static final int PARQUET_MAX_SCHEMAS_DEFAULT_VALUE = 5;
+    public static final int PARQUET_MAX_SCHEMAS_MAX_VALUE = 10;
 
     static {
-        WRITER_SUPPORTED_FORMATS = Set.of(FORMAT_JSON_LOWER_CASE);
+        WRITER_SUPPORTED_FORMATS = Set.of(FORMAT_JSON_LOWER_CASE, FORMAT_PARQUET, FORMAT_CSV_LOWER_CASE);
         WRITER_SUPPORTED_ADAPTERS = Set.of(ALIAS_LOCALFS_ADAPTER.toLowerCase(), KEY_ADAPTER_NAME_AWS_S3.toLowerCase(),
-                KEY_ADAPTER_NAME_GCS.toLowerCase());
-        WRITER_SUPPORTED_COMPRESSION = Set.of(KEY_COMPRESSION_GZIP);
+                KEY_ADAPTER_NAME_GCS.toLowerCase(), KEY_ADAPTER_NAME_HDFS.toLowerCase());
+        TEXTUAL_WRITER_SUPPORTED_COMPRESSION = Set.of(KEY_COMPRESSION_GZIP);
+        PARQUET_WRITER_SUPPORTED_COMPRESSION =
+                Set.of(KEY_COMPRESSION_GZIP, KEY_COMPRESSION_SNAPPY, KEY_COMPRESSION_ZSTD);
+        PARQUET_WRITER_SUPPORTED_VERSION = Set.of(PARQUET_WRITER_VERSION_VALUE_1, PARQUET_WRITER_VERSION_VALUE_2);
+        WRITER_SUPPORTED_QUOTES = List.of(DEFAULT_QUOTE, DEFAULT_SINGLE_QUOTE, NONE);
     }
 
     public static class DeltaOptions {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataPrefix.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataPrefix.java
index 0ed0622..3d5865a 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataPrefix.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataPrefix.java
@@ -75,7 +75,8 @@
 
     public ExternalDataPrefix(Map<String, String> configuration) throws AlgebricksException {
         String prefix = ExternalDataUtils.getDefinitionOrPath(configuration);
-        this.original = prefix != null ? prefix : "";
+        this.original = HDFSUtils.isSourceTypeHdfs(configuration) ? HDFSUtils.updateRootPath(prefix, true)
+                : (prefix != null ? prefix : "");
         this.endsWithSlash = this.original.endsWith("/");
         protocolContainerPair = ExternalDataUtils.getProtocolContainerPair(configuration);
         segments = extractPrefixSegments(original);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
index 12ce22d..6767f93 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
@@ -29,8 +29,10 @@
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_QUOTE;
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_RECORD_END;
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_RECORD_START;
+import static org.apache.asterix.external.util.aws.s3.S3AuthUtils.configureAwsS3HdfsJobConf;
 import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.validateAzureBlobProperties;
 import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.validateAzureDataLakeProperties;
+import static org.apache.asterix.external.util.google.gcs.GCSUtils.configureHdfsJobConf;
 import static org.apache.asterix.external.util.google.gcs.GCSUtils.validateProperties;
 import static org.apache.asterix.om.utils.ProjectionFiltrationTypeUtil.ALL_FIELDS_TYPE;
 import static org.apache.asterix.om.utils.ProjectionFiltrationTypeUtil.EMPTY_TYPE;
@@ -72,14 +74,15 @@
 import org.apache.asterix.external.api.IInputStreamFactory;
 import org.apache.asterix.external.api.IRecordReaderFactory;
 import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory.IncludeExcludeMatcher;
-import org.apache.asterix.external.input.record.reader.aws.delta.AwsS3DeltaReaderFactory;
 import org.apache.asterix.external.library.JavaLibrary;
 import org.apache.asterix.external.library.msgpack.MessagePackUtils;
 import org.apache.asterix.external.util.ExternalDataConstants.ParquetOptions;
+import org.apache.asterix.external.util.aws.s3.S3AuthUtils;
 import org.apache.asterix.external.util.aws.s3.S3Constants;
 import org.apache.asterix.external.util.aws.s3.S3Utils;
 import org.apache.asterix.external.util.azure.blob_storage.AzureConstants;
 import org.apache.asterix.external.util.google.gcs.GCSConstants;
+import org.apache.asterix.external.util.google.gcs.GCSUtils;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
@@ -90,6 +93,7 @@
 import org.apache.asterix.runtime.projection.ExternalDatasetProjectionFiltrationInfo;
 import org.apache.asterix.runtime.projection.FunctionCallInformation;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobConf;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -447,6 +451,7 @@
     public static void defaultConfiguration(Map<String, String> configuration) {
         String format = configuration.get(ExternalDataConstants.KEY_FORMAT);
         if (format != null) {
+            //todo:utsav
             // default quote, escape character for quote and fields delimiter for csv and tsv format
             if (format.equals(ExternalDataConstants.FORMAT_CSV)) {
                 configuration.putIfAbsent(KEY_DELIMITER, ExternalDataConstants.DEFAULT_DELIMITER);
@@ -514,21 +519,22 @@
         }
     }
 
-    public static void validateDeltaTableExists(Map<String, String> configuration) throws CompilationException {
-        Configuration conf = new Configuration();
+    public static void validateDeltaTableExists(Map<String, String> configuration) throws AlgebricksException {
         String tableMetadataPath = null;
+        JobConf conf = new JobConf();
         if (configuration.get(ExternalDataConstants.KEY_EXTERNAL_SOURCE_TYPE)
                 .equals(ExternalDataConstants.KEY_ADAPTER_NAME_AWS_S3)) {
-            AwsS3DeltaReaderFactory.applyConfiguration(configuration, conf);
-            tableMetadataPath = S3Constants.HADOOP_S3_PROTOCOL + "://"
-                    + configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME) + '/'
-                    + configuration.get(ExternalDataConstants.DEFINITION_FIELD_NAME);
+            configureAwsS3HdfsJobConf(conf, configuration);
+            tableMetadataPath = S3Utils.getPath(configuration);
+        } else if (configuration.get(ExternalDataConstants.KEY_EXTERNAL_SOURCE_TYPE)
+                .equals(ExternalDataConstants.KEY_ADAPTER_NAME_GCS)) {
+            configureHdfsJobConf(conf, configuration);
+            tableMetadataPath = GCSUtils.getPath(configuration);
         } else {
             throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR,
                     "Delta format is not supported for the external source type: "
                             + configuration.get(ExternalDataConstants.KEY_EXTERNAL_SOURCE_TYPE));
         }
-
         Engine engine = DefaultEngine.create(conf);
         io.delta.kernel.Table table = io.delta.kernel.Table.forPath(engine, tableMetadataPath);
         try {
@@ -592,7 +598,8 @@
             tableMetadataPath = S3Constants.HADOOP_S3_PROTOCOL + "://"
                     + configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME) + '/'
                     + configuration.get(ExternalDataConstants.DEFINITION_FIELD_NAME);
-        } else if (configuration.get(ExternalDataConstants.KEY_READER).equals(ExternalDataConstants.READER_HDFS)) {
+        } else if (configuration.get(ExternalDataConstants.KEY_READER)
+                .equals(ExternalDataConstants.KEY_ADAPTER_NAME_HDFS)) {
             conf.set(ExternalDataConstants.KEY_HADOOP_FILESYSTEM_URI,
                     configuration.get(ExternalDataConstants.KEY_HDFS_URL));
             tableMetadataPath = configuration.get(ExternalDataConstants.KEY_HDFS_URL) + '/' + tableMetadataPath;
@@ -633,12 +640,22 @@
     public static void validate(Map<String, String> configuration) throws HyracksDataException {
         String format = configuration.get(ExternalDataConstants.KEY_FORMAT);
         String header = configuration.get(ExternalDataConstants.KEY_HEADER);
+        String forceQuote = configuration.get(ExternalDataConstants.KEY_FORCE_QUOTE);
+        String emptyFieldAsNull = configuration.get(ExternalDataConstants.KEY_EMPTY_FIELD_AS_NULL);
         if (format != null && isHeaderRequiredFor(format) && header == null) {
             throw new RuntimeDataException(ErrorCode.PARAMETERS_REQUIRED, ExternalDataConstants.KEY_HEADER);
         }
         if (header != null && !isBoolean(header)) {
             throw new RuntimeDataException(ErrorCode.INVALID_REQ_PARAM_VAL, ExternalDataConstants.KEY_HEADER, header);
         }
+        if (forceQuote != null && !isBoolean(forceQuote)) {
+            throw new RuntimeDataException(ErrorCode.INVALID_REQ_PARAM_VAL, ExternalDataConstants.KEY_FORCE_QUOTE,
+                    forceQuote);
+        }
+        if (emptyFieldAsNull != null && !isBoolean(emptyFieldAsNull)) {
+            throw new RuntimeDataException(ErrorCode.INVALID_REQ_PARAM_VAL,
+                    ExternalDataConstants.KEY_EMPTY_FIELD_AS_NULL, emptyFieldAsNull);
+        }
         char delimiter = validateGetDelimiter(configuration);
         validateGetQuote(configuration, delimiter);
         validateGetEscape(configuration, format);
@@ -698,7 +715,7 @@
 
         switch (type) {
             case ExternalDataConstants.KEY_ADAPTER_NAME_AWS_S3:
-                S3Utils.validateProperties(configuration, srcLoc, collector);
+                S3AuthUtils.validateProperties(appCtx, configuration, srcLoc, collector);
                 break;
             case ExternalDataConstants.KEY_ADAPTER_NAME_AZURE_BLOB:
                 validateAzureBlobProperties(configuration, srcLoc, collector, appCtx);
@@ -709,6 +726,9 @@
             case ExternalDataConstants.KEY_ADAPTER_NAME_GCS:
                 validateProperties(configuration, srcLoc, collector);
                 break;
+            case ExternalDataConstants.KEY_ADAPTER_NAME_HDFS:
+                HDFSUtils.validateProperties(configuration, srcLoc, collector);
+                break;
             default:
                 // Nothing needs to be done
                 break;
@@ -1123,9 +1143,9 @@
                 String[] nodePathPair = path.trim().split("://");
                 protocol = nodePathPair[0];
                 break;
-            case ExternalDataConstants.KEY_HDFS_URL:
-                protocol = ExternalDataConstants.KEY_HDFS_URL;
-                break;
+            case ExternalDataConstants.KEY_ADAPTER_NAME_HDFS:
+                // Remove trailing slashes as prefixes/paths in hdfs start with a slash (absolute paths)
+                return configurations.get(ExternalDataConstants.KEY_HDFS_URL).replaceAll("/+$", "");
             default:
                 return "";
         }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
index f7638b4..7c7e031 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
@@ -18,8 +18,11 @@
  */
 package org.apache.asterix.external.util;
 
+import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.external.util.ExternalDataUtils.validateIncludeExclude;
 import static org.apache.asterix.om.utils.ProjectionFiltrationTypeUtil.ALL_FIELDS_TYPE;
 import static org.apache.asterix.om.utils.ProjectionFiltrationTypeUtil.EMPTY_TYPE;
+import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -27,14 +30,29 @@
 import java.io.DataOutputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.nio.file.Files;
+import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Base64;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
+
+import javax.security.auth.Subject;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.LoginContext;
+import javax.security.auth.login.LoginException;
 
 import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.external.indexing.ExternalFile;
@@ -45,29 +63,41 @@
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.runtime.projection.ExternalDatasetProjectionFiltrationInfo;
 import org.apache.asterix.runtime.projection.FunctionCallInformation;
+import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.compress.GzipCodec;
 import org.apache.hadoop.mapred.FileSplit;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
 import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.util.KerberosName;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.application.ICCServiceContext;
 import org.apache.hyracks.api.context.ICCContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.HyracksException;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.exceptions.Warning;
 import org.apache.hyracks.api.network.INetworkSecurityManager;
 import org.apache.hyracks.hdfs.scheduler.Scheduler;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 import org.apache.parquet.hadoop.ParquetInputFormat;
 
 public class HDFSUtils {
 
+    private static final Logger LOGGER = LogManager.getLogger();
+
     private HDFSUtils() {
     }
 
@@ -173,6 +203,8 @@
                 return ExternalDataConstants.CLASS_NAME_SEQUENCE_INPUT_FORMAT;
             case ExternalDataConstants.INPUT_FORMAT_PARQUET:
                 return ExternalDataConstants.CLASS_NAME_PARQUET_INPUT_FORMAT;
+            case ExternalDataConstants.INPUT_FORMAT_AVRO:
+                return ExternalDataConstants.CLASS_NAME_AVRO_INPUT_FORMAT;
             default:
                 return inputFormatParameter;
         }
@@ -204,7 +236,13 @@
                     ExternalDataConstants.CLASS_NAME_HDFS_FILESYSTEM);
             conf.set(ExternalDataConstants.KEY_HADOOP_FILESYSTEM_URI, url);
         }
-        conf.set(ExternalDataConstants.KEY_HADOOP_INPUT_DIR, configuration.get(ExternalDataConstants.KEY_PATH).trim());
+
+        // In case KEY_PATH is not being used, it means DEFINITION_FIELD_NAME is being used,
+        // and the property KEY_HADOOP_INPUT_DIR will be set in HDFSDatasourceFactory based on the provided definition.
+        String path = configuration.get(ExternalDataConstants.KEY_PATH);
+        if (path != null) {
+            HDFSUtils.setInputDir(conf, path.trim());
+        }
         conf.setClassLoader(HDFSInputStream.class.getClassLoader());
         conf.set(ExternalDataConstants.KEY_HADOOP_INPUT_FORMAT, formatClassName);
 
@@ -216,11 +254,169 @@
 
         if (ExternalDataConstants.CLASS_NAME_PARQUET_INPUT_FORMAT.equals(formatClassName)) {
             configureParquet(configuration, conf);
+        } else if (ExternalDataConstants.CLASS_NAME_AVRO_INPUT_FORMAT.equals(formatClassName)) {
+            conf.set(ExternalDataConstants.HDFS_AVRO_IGNORE_INPUTS_WITHOUT_EXTENSION, ExternalDataConstants.FALSE);
+            disableHadoopFileSystemCache(conf, ExternalDataConstants.READER_HDFS);
         }
 
+        if (configuration.containsKey(ExternalDataConstants.S3A_CHANGE_DETECTION_REQUIRED)) {
+            conf.set(ExternalDataConstants.S3A_CHANGE_DETECTION_REQUIRED_CONFIG_KEY,
+                    configuration.get(ExternalDataConstants.S3A_CHANGE_DETECTION_REQUIRED));
+        }
+
+        String useDatanodeHostname = configuration.get(ExternalDataConstants.HDFS_USE_DATANODE_HOSTNAME);
+        if (useDatanodeHostname != null) {
+            conf.set(ExternalDataConstants.KEY_HDFS_USE_DATANODE_HOSTNAME, useDatanodeHostname);
+        }
+        conf.set(ExternalDataConstants.HDFS_IO_COMPRESSION_CODECS_KEY, AliasGzipCodec.class.getName());
         return conf;
     }
 
+    public static Configuration configureHDFSwrite(Map<String, String> configuration) {
+        Configuration conf = new Configuration();
+        String url = configuration.get(ExternalDataConstants.KEY_HDFS_URL);
+        String blocksize = configuration.get(ExternalDataConstants.HDFS_BLOCKSIZE);
+        String replication = configuration.get(ExternalDataConstants.HDFS_REPLICATION);
+        String useDatanodeHostname = configuration.get(ExternalDataConstants.HDFS_USE_DATANODE_HOSTNAME);
+        if (url != null) {
+            conf.set(ExternalDataConstants.KEY_HADOOP_FILESYSTEM_URI, url);
+        }
+        if (blocksize != null) {
+            conf.set(ExternalDataConstants.KEY_HDFS_BLOCKSIZE, blocksize);
+        }
+        if (replication != null) {
+            conf.set(ExternalDataConstants.KEY_HDFS_REPLICATION, replication);
+        }
+        if (useDatanodeHostname != null) {
+            conf.set(ExternalDataConstants.KEY_HDFS_USE_DATANODE_HOSTNAME, useDatanodeHostname);
+        }
+        return conf;
+    }
+
+    public synchronized static Credentials configureHadoopAuthentication(Map<String, String> configuration,
+            Configuration conf) throws AlgebricksException {
+        UserGroupInformation.reset();
+        if (Objects.equals(configuration.get(ExternalDataConstants.HADOOP_AUTHENTICATION),
+                ExternalDataConstants.KERBEROS_PROTOCOL)) {
+            conf.set(ExternalDataConstants.KEY_HADOOP_AUTHENTICATION, ExternalDataConstants.KERBEROS_PROTOCOL);
+            conf.set(ExternalDataConstants.KEY_NAMENODE_PRINCIPAL_PATTERN, "*");
+
+            String kerberosRealm = configuration.get(ExternalDataConstants.KERBEROS_REALM);
+            String kerberosKdc = configuration.get(ExternalDataConstants.KERBEROS_KDC);
+            String kerberosPrincipal = configuration.get(ExternalDataConstants.KERBEROS_PRINCIPAL);
+            String kerberosPassword = configuration.get(ExternalDataConstants.KERBEROS_PASSWORD);
+
+            javax.security.auth.login.Configuration config = new JaasConfiguration();
+            java.nio.file.Path krb5conf = null;
+            try {
+                krb5conf = createKerberosConf(kerberosRealm, kerberosKdc);
+                System.setProperty(ExternalDataConstants.KEY_KERBEROS_CONF, krb5conf.toString());
+
+                Subject subject = new Subject();
+                LoginContext loginContext =
+                        new LoginContext("", subject, new callbackHandler(kerberosPrincipal, kerberosPassword), config);
+                loginContext.login();
+
+                // Hadoop libraries use rules to ensure that a principal is associated with a realm.
+                // The default realm is static and needs to be modified in order to work with a different realm.
+                KerberosName.resetDefaultRealm();
+                UserGroupInformation.setConfiguration(conf);
+                UserGroupInformation ugi = UserGroupInformation.getUGIFromSubject(subject);
+                Credentials credentials = new Credentials();
+                ugi.doAs((PrivilegedExceptionAction<Void>) () -> {
+                    try (FileSystem fs = FileSystem.get(conf)) {
+                        fs.addDelegationTokens(ugi.getUserName(), credentials);
+                    }
+                    return null;
+                });
+                loginContext.logout();
+                return credentials;
+            } catch (InterruptedException ex) {
+                Thread.currentThread().interrupt();
+                throw CompilationException.create(ErrorCode.COULD_NOT_CREATE_TOKENS);
+            } catch (LoginException | IOException ex) {
+                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex, getMessageOrToString(ex));
+            } finally {
+                System.clearProperty(ExternalDataConstants.KEY_KERBEROS_CONF);
+                UserGroupInformation.reset();
+                if (krb5conf != null) {
+                    FileUtils.deleteQuietly(krb5conf.toFile());
+                }
+            }
+        }
+        return null;
+    }
+
+    private static java.nio.file.Path createKerberosConf(String kerberosRealm, String kerberosKdc)
+            throws AlgebricksException {
+        java.nio.file.Path krb5conf;
+        try {
+            krb5conf = Files.createTempFile(ExternalDataConstants.KERBEROS_CONFIG_FILE_PATTERN[0],
+                    ExternalDataConstants.KERBEROS_CONFIG_FILE_PATTERN[1]);
+            Files.writeString(krb5conf,
+                    String.format(ExternalDataConstants.KERBEROS_CONFIG_FILE_CONTENT, kerberosRealm, kerberosKdc));
+        } catch (IOException ex) {
+            LOGGER.error("Failed to create kerberos config file", ex);
+            throw AsterixException.create(ErrorCode.COULD_NOT_CREATE_TOKENS);
+        }
+        return krb5conf;
+    }
+
+    private static class JaasConfiguration extends javax.security.auth.login.Configuration {
+        @Override
+        public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
+            Map<String, Object> options =
+                    Map.of(ExternalDataConstants.KERBEROS_CONFIG_REFRESH, ExternalDataConstants.TRUE);
+
+            return new AppConfigurationEntry[] { new AppConfigurationEntry(ExternalDataConstants.KERBEROS_LOGIN_MODULE,
+                    AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, options) };
+        }
+    }
+
+    public static byte[] serialize(Writable writable) throws IOException {
+        try (ByteArrayOutputStream bos = new ByteArrayOutputStream();
+                DataOutputStream oos = new DataOutputStream(bos)) {
+            writable.write(oos);
+            return bos.toByteArray();
+        }
+    }
+
+    public static void deserialize(byte[] data, Writable writable) throws IOException {
+        try (DataInputStream din = new DataInputStream(new ByteArrayInputStream(data))) {
+            writable.readFields(din);
+        }
+    }
+
+    private static class callbackHandler implements CallbackHandler {
+        private final String principal;
+        private final String password;
+
+        public callbackHandler(String principal, String password) {
+            this.principal = principal;
+            this.password = password;
+        }
+
+        @Override
+        public void handle(Callback[] callbacks) throws UnsupportedCallbackException {
+            for (Callback callback : callbacks) {
+                if (callback instanceof NameCallback) {
+                    ((NameCallback) callback).setName(principal);
+                } else if (callback instanceof PasswordCallback) {
+                    ((PasswordCallback) callback).setPassword(password.toCharArray());
+                } else {
+                    throw new UnsupportedCallbackException(callback);
+                }
+            }
+        }
+    }
+
+    public static String updateRootPath(String path, boolean updateNullPath) {
+        if (path != null && path.isEmpty()) {
+            return "/";
+        }
+        return path != null ? path : (updateNullPath ? "/" : null);
+    }
+
     private static void configureParquet(Map<String, String> configuration, JobConf conf) {
         //Parquet configurations
         conf.set(ParquetInputFormat.READ_SUPPORT_CLASS, ParquetReadSupport.class.getName());
@@ -351,4 +547,64 @@
     public static boolean isEmpty(JobConf job) {
         return job.get(ExternalDataConstants.KEY_HADOOP_INPUT_DIR, "").isEmpty();
     }
+
+    public static void setInputDir(JobConf conf, String path) {
+        conf.set(ExternalDataConstants.KEY_HADOOP_INPUT_DIR, path);
+    }
+
+    public static void validateProperties(Map<String, String> configuration, SourceLocation srcLoc,
+            IWarningCollector collector) throws CompilationException {
+        if (configuration.get(ExternalDataConstants.KEY_HDFS_URL) == null) {
+            throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_HDFS_URL);
+        }
+        if (configuration.get(ExternalDataConstants.KEY_INPUT_FORMAT) == null) {
+            throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc,
+                    ExternalDataConstants.KEY_INPUT_FORMAT);
+        }
+
+        if (Objects.equals(configuration.get(ExternalDataConstants.HADOOP_AUTHENTICATION),
+                ExternalDataConstants.KERBEROS_PROTOCOL)) {
+            String kerberosRealm = configuration.get(ExternalDataConstants.KERBEROS_REALM);
+            String kerberosKdc = configuration.get(ExternalDataConstants.KERBEROS_KDC);
+            String kerberosPrincipal = configuration.get(ExternalDataConstants.KERBEROS_PRINCIPAL);
+            String kerberosPassword = configuration.get(ExternalDataConstants.KERBEROS_PASSWORD);
+
+            if (kerberosRealm == null) {
+                throw CompilationException.create(REQUIRED_PARAM_IF_PARAM_IS_PRESENT,
+                        ExternalDataConstants.KERBEROS_REALM, ExternalDataConstants.HADOOP_AUTHENTICATION);
+            }
+            if (kerberosKdc == null) {
+                throw CompilationException.create(REQUIRED_PARAM_IF_PARAM_IS_PRESENT,
+                        ExternalDataConstants.KERBEROS_KDC, ExternalDataConstants.HADOOP_AUTHENTICATION);
+            }
+            if (kerberosPrincipal == null) {
+                throw CompilationException.create(REQUIRED_PARAM_IF_PARAM_IS_PRESENT,
+                        ExternalDataConstants.KERBEROS_PRINCIPAL, ExternalDataConstants.HADOOP_AUTHENTICATION);
+            }
+            if (kerberosPassword == null) {
+                throw CompilationException.create(REQUIRED_PARAM_IF_PARAM_IS_PRESENT,
+                        ExternalDataConstants.KERBEROS_PASSWORD, ExternalDataConstants.HADOOP_AUTHENTICATION);
+            }
+        }
+
+        // For validation purposes for external data prefixes
+        validateIncludeExclude(configuration);
+        try {
+            new ExternalDataPrefix(configuration);
+        } catch (AlgebricksException ex) {
+            throw new CompilationException(ErrorCode.FAILED_TO_CALCULATE_COMPUTED_FIELDS, ex);
+        }
+    }
+
+    public static boolean isSourceTypeHdfs(Map<String, String> configuration) {
+        return ExternalDataConstants.KEY_ADAPTER_NAME_HDFS
+                .equalsIgnoreCase(configuration.get(ExternalDataConstants.KEY_EXTERNAL_SOURCE_TYPE));
+    }
+
+    public static class AliasGzipCodec extends GzipCodec {
+        @Override
+        public String getDefaultExtension() {
+            return "." + ExternalDataConstants.KEY_COMPRESSION_GZIP;
+        }
+    }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/WriterValidationUtil.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/WriterValidationUtil.java
index 843600e..380a9a8 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/WriterValidationUtil.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/WriterValidationUtil.java
@@ -19,8 +19,18 @@
 package org.apache.asterix.external.util;
 
 import static org.apache.asterix.common.exceptions.ErrorCode.INVALID_REQ_PARAM_VAL;
+import static org.apache.asterix.common.exceptions.ErrorCode.MAXIMUM_VALUE_ALLOWED_FOR_PARAM;
 import static org.apache.asterix.common.exceptions.ErrorCode.MINIMUM_VALUE_ALLOWED_FOR_PARAM;
+import static org.apache.asterix.common.exceptions.ErrorCode.PARAMETERS_REQUIRED;
+import static org.apache.asterix.external.util.ExternalDataConstants.FORMAT_CSV;
+import static org.apache.asterix.external.util.ExternalDataConstants.FORMAT_JSON_LOWER_CASE;
+import static org.apache.asterix.external.util.ExternalDataConstants.FORMAT_PARQUET;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_PARQUET_PAGE_SIZE;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_PARQUET_ROW_GROUP_SIZE;
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_WRITER_MAX_RESULT;
+import static org.apache.asterix.external.util.ExternalDataConstants.PARQUET_MAX_SCHEMAS_KEY;
+import static org.apache.asterix.external.util.ExternalDataConstants.PARQUET_MAX_SCHEMAS_MAX_VALUE;
+import static org.apache.asterix.external.util.ExternalDataConstants.PARQUET_WRITER_VERSION_KEY;
 import static org.apache.asterix.external.util.ExternalDataConstants.WRITER_MAX_RESULT_MINIMUM;
 
 import java.util.List;
@@ -31,6 +41,7 @@
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.util.StorageUtil;
 
 public class WriterValidationUtil {
 
@@ -41,10 +52,18 @@
             Map<String, String> configuration, SourceLocation sourceLocation) throws CompilationException {
         validateAdapter(adapter, supportedAdapters, sourceLocation);
         validateFormat(configuration, sourceLocation);
-        validateCompression(configuration, sourceLocation);
         validateMaxResult(configuration, sourceLocation);
     }
 
+    private static void validateQuote(Map<String, String> configuration, SourceLocation sourceLocation)
+            throws CompilationException {
+        String quote = configuration.get(ExternalDataConstants.KEY_QUOTE);
+        if (quote != null && !ExternalDataConstants.WRITER_SUPPORTED_QUOTES.contains(quote.toLowerCase())) {
+            throw CompilationException.create(ErrorCode.INVALID_QUOTE, sourceLocation, quote,
+                    ExternalDataConstants.WRITER_SUPPORTED_QUOTES.toString());
+        }
+    }
+
     private static void validateAdapter(String adapter, Set<String> supportedAdapters, SourceLocation sourceLocation)
             throws CompilationException {
         checkSupported(ExternalDataConstants.KEY_EXTERNAL_SOURCE_TYPE, adapter, supportedAdapters,
@@ -56,14 +75,87 @@
         String format = configuration.get(ExternalDataConstants.KEY_FORMAT);
         checkSupported(ExternalDataConstants.KEY_FORMAT, format, ExternalDataConstants.WRITER_SUPPORTED_FORMATS,
                 ErrorCode.UNSUPPORTED_WRITING_FORMAT, sourceLocation, false);
+        switch (format.toLowerCase()) {
+            case FORMAT_JSON_LOWER_CASE:
+                validateJSON(configuration, sourceLocation);
+                break;
+            case FORMAT_PARQUET:
+                validateParquet(configuration, sourceLocation);
+                break;
+            case FORMAT_CSV:
+                validateCSV(configuration, sourceLocation);
+                break;
+        }
     }
 
-    private static void validateCompression(Map<String, String> configuration, SourceLocation sourceLocation)
+    private static void validateParquet(Map<String, String> configuration, SourceLocation sourceLocation)
+            throws CompilationException {
+        validateParquetCompression(configuration, sourceLocation);
+        validateParquetRowGroupSize(configuration);
+        validateParquetPageSize(configuration);
+        validateVersion(configuration, sourceLocation);
+        validateMaxParquetSchemas(configuration, sourceLocation);
+    }
+
+    private static void validateVersion(Map<String, String> configuration, SourceLocation sourceLocation)
+            throws CompilationException {
+        String version = configuration.get(PARQUET_WRITER_VERSION_KEY);
+        checkSupported(PARQUET_WRITER_VERSION_KEY, version, ExternalDataConstants.PARQUET_WRITER_SUPPORTED_VERSION,
+                ErrorCode.INVALID_PARQUET_WRITER_VERSION, sourceLocation, true);
+    }
+
+    private static void validateParquetRowGroupSize(Map<String, String> configuration) throws CompilationException {
+        String rowGroupSize = configuration.get(KEY_PARQUET_ROW_GROUP_SIZE);
+        if (rowGroupSize == null)
+            return;
+        try {
+            StorageUtil.getByteValue(rowGroupSize);
+        } catch (IllegalArgumentException e) {
+            throw CompilationException.create(ErrorCode.ILLEGAL_SIZE_PROVIDED, KEY_PARQUET_ROW_GROUP_SIZE,
+                    rowGroupSize);
+        }
+    }
+
+    private static void validateParquetPageSize(Map<String, String> configuration) throws CompilationException {
+        String pageSize = configuration.get(KEY_PARQUET_PAGE_SIZE);
+        if (pageSize == null)
+            return;
+        try {
+            StorageUtil.getByteValue(pageSize);
+        } catch (IllegalArgumentException e) {
+            throw CompilationException.create(ErrorCode.ILLEGAL_SIZE_PROVIDED, KEY_PARQUET_PAGE_SIZE, pageSize);
+        }
+    }
+
+    private static void validateJSON(Map<String, String> configuration, SourceLocation sourceLocation)
+            throws CompilationException {
+        validateTextualCompression(configuration, sourceLocation);
+    }
+
+    private static void validateCSV(Map<String, String> configuration, SourceLocation sourceLocation)
+            throws CompilationException {
+        validateTextualCompression(configuration, sourceLocation);
+        validateDelimiter(configuration, sourceLocation);
+        validateRecordDelimiter(configuration, sourceLocation);
+        validateQuote(configuration, sourceLocation);
+        validateEscape(configuration, sourceLocation);
+    }
+
+    private static void validateParquetCompression(Map<String, String> configuration, SourceLocation sourceLocation)
             throws CompilationException {
         String compression = configuration.get(ExternalDataConstants.KEY_WRITER_COMPRESSION);
-        checkSupported(ExternalDataConstants.KEY_WRITER_COMPRESSION, compression,
-                ExternalDataConstants.WRITER_SUPPORTED_COMPRESSION, ErrorCode.UNKNOWN_COMPRESSION_SCHEME,
-                sourceLocation, true);
+        checkCompressionSupported(ExternalDataConstants.KEY_WRITER_COMPRESSION, compression,
+                ExternalDataConstants.PARQUET_WRITER_SUPPORTED_COMPRESSION,
+                ErrorCode.UNSUPPORTED_WRITER_COMPRESSION_SCHEME, sourceLocation, FORMAT_PARQUET, true);
+    }
+
+    private static void validateTextualCompression(Map<String, String> configuration, SourceLocation sourceLocation)
+            throws CompilationException {
+        String compression = configuration.get(ExternalDataConstants.KEY_WRITER_COMPRESSION);
+        checkCompressionSupported(ExternalDataConstants.KEY_WRITER_COMPRESSION, compression,
+                ExternalDataConstants.TEXTUAL_WRITER_SUPPORTED_COMPRESSION,
+                ErrorCode.UNSUPPORTED_WRITER_COMPRESSION_SCHEME, sourceLocation,
+                configuration.get(ExternalDataConstants.KEY_FORMAT), true);
         if (ExternalDataUtils.isGzipCompression(compression)) {
             validateGzipCompressionLevel(configuration, sourceLocation);
         }
@@ -87,6 +179,24 @@
         }
     }
 
+    private static void validateMaxParquetSchemas(Map<String, String> configuration, SourceLocation sourceLocation)
+            throws CompilationException {
+        String maxResult = configuration.get(PARQUET_MAX_SCHEMAS_KEY);
+        if (maxResult == null) {
+            return;
+        }
+
+        try {
+            int value = Integer.parseInt(maxResult);
+            if (value > PARQUET_MAX_SCHEMAS_MAX_VALUE) {
+                throw new CompilationException(MAXIMUM_VALUE_ALLOWED_FOR_PARAM, PARQUET_MAX_SCHEMAS_KEY,
+                        PARQUET_MAX_SCHEMAS_MAX_VALUE, value);
+            }
+        } catch (NumberFormatException e) {
+            throw CompilationException.create(ErrorCode.INTEGER_VALUE_EXPECTED, sourceLocation, maxResult);
+        }
+    }
+
     private static void checkSupported(String paramKey, String value, Set<String> supportedSet, ErrorCode errorCode,
             SourceLocation sourceLocation, boolean optional) throws CompilationException {
         if (optional && value == null) {
@@ -94,7 +204,7 @@
         }
 
         if (value == null) {
-            throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, sourceLocation, paramKey);
+            throw new CompilationException(PARAMETERS_REQUIRED, sourceLocation, paramKey);
         }
 
         String normalizedValue = value.toLowerCase();
@@ -121,4 +231,49 @@
         }
     }
 
+    private static void checkCompressionSupported(String paramKey, String value, Set<String> supportedSet,
+            ErrorCode errorCode, SourceLocation sourceLocation, String format, boolean optional)
+            throws CompilationException {
+        if (optional && value == null) {
+            return;
+        }
+
+        if (value == null) {
+            throw new CompilationException(PARAMETERS_REQUIRED, sourceLocation, paramKey);
+        }
+
+        String normalizedValue = value.toLowerCase();
+        if (!supportedSet.contains(normalizedValue)) {
+            List<String> sorted = supportedSet.stream().sorted().collect(Collectors.toList());
+            throw CompilationException.create(errorCode, sourceLocation, value, format, sorted.toString());
+        }
+    }
+
+    private static void validateDelimiter(Map<String, String> configuration, SourceLocation sourceLocation)
+            throws CompilationException {
+        // Will this affect backward compatibility
+        String delimiter = configuration.get(ExternalDataConstants.KEY_DELIMITER);
+        unitByteCondition(delimiter, sourceLocation, ErrorCode.INVALID_DELIMITER);
+    }
+
+    private static void validateEscape(Map<String, String> configuration, SourceLocation sourceLocation)
+            throws CompilationException {
+        // Will this affect backward compatibility?
+        String escape = configuration.get(ExternalDataConstants.KEY_ESCAPE);
+        unitByteCondition(escape, sourceLocation, ErrorCode.INVALID_ESCAPE);
+    }
+
+    private static void validateRecordDelimiter(Map<String, String> configuration, SourceLocation sourceLocation)
+            throws CompilationException {
+        String recordDel = configuration.get(ExternalDataConstants.KEY_RECORD_DELIMITER);
+        unitByteCondition(recordDel, sourceLocation, ErrorCode.INVALID_FORCE_QUOTE);
+    }
+
+    private static void unitByteCondition(String param, SourceLocation sourceLocation, ErrorCode errorCode)
+            throws CompilationException {
+        if (param != null && param.length() > 1 && param.getBytes().length != 1) {
+            throw CompilationException.create(errorCode, sourceLocation, param);
+        }
+    }
+
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3AuthUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3AuthUtils.java
new file mode 100644
index 0000000..45988e8
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3AuthUtils.java
@@ -0,0 +1,486 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util.aws.s3;
+
+import static org.apache.asterix.common.exceptions.ErrorCode.INVALID_PARAM_VALUE_ALLOWED_VALUE;
+import static org.apache.asterix.common.exceptions.ErrorCode.PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.common.exceptions.ErrorCode.S3_REGION_NOT_SUPPORTED;
+import static org.apache.asterix.external.util.ExternalDataUtils.getPrefix;
+import static org.apache.asterix.external.util.ExternalDataUtils.isDeltaTable;
+import static org.apache.asterix.external.util.ExternalDataUtils.validateDeltaTableExists;
+import static org.apache.asterix.external.util.ExternalDataUtils.validateDeltaTableProperties;
+import static org.apache.asterix.external.util.ExternalDataUtils.validateIncludeExclude;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.ACCESS_KEY_ID_FIELD_NAME;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.ERROR_INTERNAL_ERROR;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.ERROR_METHOD_NOT_IMPLEMENTED;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.ERROR_SLOW_DOWN;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.EXTERNAL_ID_FIELD_NAME;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.HADOOP_ACCESS_KEY_ID;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.HADOOP_ANONYMOUS_ACCESS;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.HADOOP_CREDENTIAL_PROVIDER_KEY;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.HADOOP_PATH_STYLE_ACCESS;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.HADOOP_S3_CONNECTION_POOL_SIZE;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.HADOOP_S3_PROTOCOL;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.HADOOP_SECRET_ACCESS_KEY;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.HADOOP_SERVICE_END_POINT;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.HADOOP_SESSION_TOKEN;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.HADOOP_TEMP_ACCESS;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.INSTANCE_PROFILE_FIELD_NAME;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.REGION_FIELD_NAME;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.ROLE_ARN_FIELD_NAME;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.SECRET_ACCESS_KEY_FIELD_NAME;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.SERVICE_END_POINT_FIELD_NAME;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.SESSION_TOKEN_FIELD_NAME;
+import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+
+import org.apache.asterix.common.api.IApplicationContext;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.external.IExternalCredentialsCache;
+import org.apache.asterix.common.external.IExternalCredentialsCacheUpdater;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataPrefix;
+import org.apache.asterix.external.util.HDFSUtils;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.hyracks.api.util.CleanupUtils;
+
+import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
+import software.amazon.awssdk.auth.credentials.InstanceProfileCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.exception.SdkException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.awssdk.services.s3.S3ClientBuilder;
+import software.amazon.awssdk.services.s3.model.ListObjectsResponse;
+import software.amazon.awssdk.services.s3.model.ListObjectsV2Response;
+import software.amazon.awssdk.services.s3.model.S3Exception;
+import software.amazon.awssdk.services.s3.model.S3Response;
+import software.amazon.awssdk.services.sts.StsClient;
+import software.amazon.awssdk.services.sts.model.AssumeRoleRequest;
+import software.amazon.awssdk.services.sts.model.AssumeRoleResponse;
+import software.amazon.awssdk.services.sts.model.Credentials;
+
+public class S3AuthUtils {
+    private S3AuthUtils() {
+        throw new AssertionError("do not instantiate");
+    }
+
+    public static boolean isRetryableError(String errorCode) {
+        return errorCode.equals(ERROR_INTERNAL_ERROR) || errorCode.equals(ERROR_SLOW_DOWN);
+    }
+
+    /**
+     * Builds the S3 client using the provided configuration
+     *
+     * @param configuration properties
+     * @return S3 client
+     * @throws CompilationException CompilationException
+     */
+    public static S3Client buildAwsS3Client(IApplicationContext appCtx, Map<String, String> configuration)
+            throws CompilationException {
+        String regionId = configuration.get(REGION_FIELD_NAME);
+        String serviceEndpoint = configuration.get(SERVICE_END_POINT_FIELD_NAME);
+
+        Region region = validateAndGetRegion(regionId);
+        AwsCredentialsProvider credentialsProvider = buildCredentialsProvider(appCtx, configuration);
+
+        S3ClientBuilder builder = S3Client.builder();
+        builder.region(region);
+        builder.crossRegionAccessEnabled(true);
+        builder.credentialsProvider(credentialsProvider);
+
+        // Validate the service endpoint if present
+        if (serviceEndpoint != null) {
+            try {
+                URI uri = new URI(serviceEndpoint);
+                try {
+                    builder.endpointOverride(uri);
+                } catch (NullPointerException ex) {
+                    throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex, getMessageOrToString(ex));
+                }
+            } catch (URISyntaxException ex) {
+                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex,
+                        String.format("Invalid service endpoint %s", serviceEndpoint));
+            }
+        }
+
+        return builder.build();
+    }
+
+    public static AwsCredentialsProvider buildCredentialsProvider(IApplicationContext appCtx,
+            Map<String, String> configuration) throws CompilationException {
+        String arnRole = configuration.get(ROLE_ARN_FIELD_NAME);
+        String externalId = configuration.get(EXTERNAL_ID_FIELD_NAME);
+        String instanceProfile = configuration.get(INSTANCE_PROFILE_FIELD_NAME);
+        String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
+        String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
+
+        if (noAuth(configuration)) {
+            return AnonymousCredentialsProvider.create();
+        } else if (arnRole != null) {
+            return getTrustAccountCredentials(appCtx, configuration);
+        } else if (instanceProfile != null) {
+            return getInstanceProfileCredentials(configuration);
+        } else if (accessKeyId != null || secretAccessKey != null) {
+            return getAccessKeyCredentials(configuration);
+        } else {
+            if (externalId != null) {
+                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ROLE_ARN_FIELD_NAME,
+                        EXTERNAL_ID_FIELD_NAME);
+            } else {
+                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
+                        SESSION_TOKEN_FIELD_NAME);
+            }
+        }
+    }
+
+    public static Region validateAndGetRegion(String regionId) throws CompilationException {
+        List<Region> regions = S3Client.serviceMetadata().regions();
+        Optional<Region> selectedRegion = regions.stream().filter(region -> region.id().equals(regionId)).findFirst();
+
+        if (selectedRegion.isEmpty()) {
+            throw new CompilationException(S3_REGION_NOT_SUPPORTED, regionId);
+        }
+        return selectedRegion.get();
+    }
+
+    private static boolean noAuth(Map<String, String> configuration) {
+        return getNonNull(configuration, INSTANCE_PROFILE_FIELD_NAME, ROLE_ARN_FIELD_NAME, EXTERNAL_ID_FIELD_NAME,
+                ACCESS_KEY_ID_FIELD_NAME, SECRET_ACCESS_KEY_FIELD_NAME, SESSION_TOKEN_FIELD_NAME) == null;
+    }
+
+    /**
+     * Returns the cached credentials if valid, otherwise, generates new credentials by assume a role
+     *
+     * @param appCtx application context
+     * @param configuration configuration
+     * @return returns the cached credentials if valid, otherwise, generates new credentials by assume a role
+     * @throws CompilationException CompilationException
+     */
+    public static AwsCredentialsProvider getTrustAccountCredentials(IApplicationContext appCtx,
+            Map<String, String> configuration) throws CompilationException {
+        IExternalCredentialsCache cache = appCtx.getExternalCredentialsCache();
+        Object credentialsObject = cache.getCredentials(configuration);
+        if (credentialsObject != null) {
+            return () -> (AwsSessionCredentials) credentialsObject;
+        }
+        IExternalCredentialsCacheUpdater cacheUpdater = appCtx.getExternalCredentialsCacheUpdater();
+        AwsSessionCredentials credentials;
+        try {
+            credentials = (AwsSessionCredentials) cacheUpdater.generateAndCacheCredentials(configuration);
+        } catch (HyracksDataException ex) {
+            throw new CompilationException(ErrorCode.FAILED_EXTERNAL_CROSS_ACCOUNT_AUTHENTICATION, ex, ex.getMessage());
+        }
+
+        return () -> credentials;
+    }
+
+    /**
+     * Assume role using provided credentials and return the new credentials
+     *
+     * @param configuration configuration
+     * @return return credentials from the assume role
+     * @throws CompilationException CompilationException
+     */
+    public static AwsCredentialsProvider assumeRoleAndGetCredentials(Map<String, String> configuration)
+            throws CompilationException {
+        String regionId = configuration.get(REGION_FIELD_NAME);
+        String arnRole = configuration.get(ROLE_ARN_FIELD_NAME);
+        String externalId = configuration.get(EXTERNAL_ID_FIELD_NAME);
+        Region region = validateAndGetRegion(regionId);
+
+        AssumeRoleRequest.Builder builder = AssumeRoleRequest.builder();
+        builder.roleArn(arnRole);
+        builder.roleSessionName(UUID.randomUUID().toString());
+        builder.durationSeconds(900); // TODO(htowaileb): configurable? Can be 900 to 43200 (15 mins to 12 hours)
+        if (externalId != null) {
+            builder.externalId(externalId);
+        }
+
+        // credentials to be used to assume the role
+        AwsCredentialsProvider credentialsProvider;
+        AssumeRoleRequest request = builder.build();
+        String instanceProfile = configuration.get(INSTANCE_PROFILE_FIELD_NAME);
+        String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
+        String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
+        if ("true".equalsIgnoreCase(instanceProfile)) {
+            credentialsProvider = getInstanceProfileCredentials(configuration, true);
+        } else if (accessKeyId != null && secretAccessKey != null) {
+            credentialsProvider = getAccessKeyCredentials(configuration, true);
+        } else {
+            throw new CompilationException(ErrorCode.NO_AWS_VALID_PARAMS_FOUND_FOR_CROSS_ACCOUNT_TRUST_AUTHENTICATION);
+        }
+
+        // assume the role from the provided arn
+        try (StsClient stsClient =
+                StsClient.builder().region(region).credentialsProvider(credentialsProvider).build()) {
+            AssumeRoleResponse response = stsClient.assumeRole(request);
+            Credentials credentials = response.credentials();
+            return StaticCredentialsProvider.create(AwsSessionCredentials.create(credentials.accessKeyId(),
+                    credentials.secretAccessKey(), credentials.sessionToken()));
+        } catch (SdkException ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex, getMessageOrToString(ex));
+        }
+    }
+
+    private static AwsCredentialsProvider getInstanceProfileCredentials(Map<String, String> configuration)
+            throws CompilationException {
+        return getInstanceProfileCredentials(configuration, false);
+    }
+
+    private static AwsCredentialsProvider getInstanceProfileCredentials(Map<String, String> configuration,
+            boolean assumeRoleAuthentication) throws CompilationException {
+        String instanceProfile = configuration.get(INSTANCE_PROFILE_FIELD_NAME);
+
+        // only "true" value is allowed
+        if (!"true".equalsIgnoreCase(instanceProfile)) {
+            throw new CompilationException(INVALID_PARAM_VALUE_ALLOWED_VALUE, INSTANCE_PROFILE_FIELD_NAME, "true");
+        }
+
+        if (!assumeRoleAuthentication) {
+            String notAllowed = getNonNull(configuration, ACCESS_KEY_ID_FIELD_NAME, SECRET_ACCESS_KEY_FIELD_NAME,
+                    SESSION_TOKEN_FIELD_NAME);
+            if (notAllowed != null) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, notAllowed,
+                        INSTANCE_PROFILE_FIELD_NAME);
+            }
+        }
+        return InstanceProfileCredentialsProvider.create();
+    }
+
+    private static AwsCredentialsProvider getAccessKeyCredentials(Map<String, String> configuration)
+            throws CompilationException {
+        return getAccessKeyCredentials(configuration, false);
+    }
+
+    private static AwsCredentialsProvider getAccessKeyCredentials(Map<String, String> configuration,
+            boolean assumeRoleAuthentication) throws CompilationException {
+        String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
+        String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
+        String sessionToken = configuration.get(SESSION_TOKEN_FIELD_NAME);
+
+        if (accessKeyId == null) {
+            throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
+                    SECRET_ACCESS_KEY_FIELD_NAME);
+        }
+        if (secretAccessKey == null) {
+            throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, SECRET_ACCESS_KEY_FIELD_NAME,
+                    ACCESS_KEY_ID_FIELD_NAME);
+        }
+
+        if (!assumeRoleAuthentication) {
+            String notAllowed = getNonNull(configuration, INSTANCE_PROFILE_FIELD_NAME, EXTERNAL_ID_FIELD_NAME);
+            if (notAllowed != null) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, notAllowed,
+                        INSTANCE_PROFILE_FIELD_NAME);
+            }
+        }
+
+        // use session token if provided
+        if (sessionToken != null) {
+            return StaticCredentialsProvider
+                    .create(AwsSessionCredentials.create(accessKeyId, secretAccessKey, sessionToken));
+        } else {
+            return StaticCredentialsProvider.create(AwsBasicCredentials.create(accessKeyId, secretAccessKey));
+        }
+    }
+
+    private static String getNonNull(Map<String, String> configuration, String... fieldNames) {
+        for (String fieldName : fieldNames) {
+            if (configuration.get(fieldName) != null) {
+                return fieldName;
+            }
+        }
+        return null;
+    }
+
+    /**
+     * Builds the S3 client using the provided configuration
+     *
+     * @param configuration      properties
+     * @param numberOfPartitions number of partitions in the cluster
+     */
+    public static void configureAwsS3HdfsJobConf(JobConf conf, Map<String, String> configuration) {
+        configureAwsS3HdfsJobConf(conf, configuration, 0);
+    }
+
+    public static void configureAwsS3HdfsJobConf(JobConf conf, Map<String, String> configuration,
+            int numberOfPartitions) {
+        String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
+        String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
+        String sessionToken = configuration.get(SESSION_TOKEN_FIELD_NAME);
+        String serviceEndpoint = configuration.get(SERVICE_END_POINT_FIELD_NAME);
+
+        //Disable caching S3 FileSystem
+        HDFSUtils.disableHadoopFileSystemCache(conf, HADOOP_S3_PROTOCOL);
+
+        /*
+         * Authentication Methods:
+         * 1- Anonymous: no accessKeyId and no secretAccessKey
+         * 2- Temporary: has to provide accessKeyId, secretAccessKey and sessionToken
+         * 3- Private: has to provide accessKeyId and secretAccessKey
+         */
+        if (accessKeyId == null) {
+            //Tells hadoop-aws it is an anonymous access
+            conf.set(HADOOP_CREDENTIAL_PROVIDER_KEY, HADOOP_ANONYMOUS_ACCESS);
+        } else {
+            conf.set(HADOOP_ACCESS_KEY_ID, accessKeyId);
+            conf.set(HADOOP_SECRET_ACCESS_KEY, secretAccessKey);
+            if (sessionToken != null) {
+                conf.set(HADOOP_SESSION_TOKEN, sessionToken);
+                //Tells hadoop-aws it is a temporary access
+                conf.set(HADOOP_CREDENTIAL_PROVIDER_KEY, HADOOP_TEMP_ACCESS);
+            }
+        }
+
+        /*
+         * This is to allow S3 definition to have path-style form. Should always be true to match the current
+         * way we access files in S3
+         */
+        conf.set(HADOOP_PATH_STYLE_ACCESS, ExternalDataConstants.TRUE);
+
+        /*
+         * Set the size of S3 connection pool to be the number of partitions
+         */
+        if (numberOfPartitions != 0) {
+            conf.set(HADOOP_S3_CONNECTION_POOL_SIZE, String.valueOf(numberOfPartitions));
+        }
+
+        if (serviceEndpoint != null) {
+            // Validation of the URL should be done at hadoop-aws level
+            conf.set(HADOOP_SERVICE_END_POINT, serviceEndpoint);
+        } else {
+            //Region is ignored and buckets could be found by the central endpoint
+            conf.set(HADOOP_SERVICE_END_POINT, Constants.CENTRAL_ENDPOINT);
+        }
+    }
+
+    /**
+     * Validate external dataset properties
+     *
+     * @param configuration properties
+     * @throws CompilationException Compilation exception
+     */
+    public static void validateProperties(IApplicationContext appCtx, Map<String, String> configuration,
+            SourceLocation srcLoc, IWarningCollector collector) throws CompilationException {
+        if (isDeltaTable(configuration)) {
+            validateDeltaTableProperties(configuration);
+        }
+        // check if the format property is present
+        else if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
+            throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
+        }
+
+        String arnRole = configuration.get(ROLE_ARN_FIELD_NAME);
+        String externalId = configuration.get(EXTERNAL_ID_FIELD_NAME);
+        String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
+        String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
+
+        if (arnRole != null) {
+            return;
+        } else if (externalId != null) {
+            throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ROLE_ARN_FIELD_NAME,
+                    EXTERNAL_ID_FIELD_NAME);
+        } else if (accessKeyId == null || secretAccessKey == null) {
+            // If one is passed, the other is required
+            if (accessKeyId != null) {
+                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, SECRET_ACCESS_KEY_FIELD_NAME,
+                        ACCESS_KEY_ID_FIELD_NAME);
+            } else if (secretAccessKey != null) {
+                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
+                        SECRET_ACCESS_KEY_FIELD_NAME);
+            }
+        }
+
+        validateIncludeExclude(configuration);
+        try {
+            // TODO(htowaileb): maybe something better, this will check to ensure type is supported before creation
+            new ExternalDataPrefix(configuration);
+        } catch (AlgebricksException ex) {
+            throw new CompilationException(ErrorCode.FAILED_TO_CALCULATE_COMPUTED_FIELDS, ex);
+        }
+
+        // Check if the bucket is present
+        S3Client s3Client = buildAwsS3Client(appCtx, configuration);
+        S3Response response;
+        boolean useOldApi = false;
+        String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+        String prefix = getPrefix(configuration);
+
+        try {
+            response = S3Utils.isBucketEmpty(s3Client, container, prefix, false);
+        } catch (S3Exception ex) {
+            // Method not implemented, try falling back to old API
+            try {
+                // For error code, see https://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html
+                if (ex.awsErrorDetails().errorCode().equals(ERROR_METHOD_NOT_IMPLEMENTED)) {
+                    useOldApi = true;
+                    response = S3Utils.isBucketEmpty(s3Client, container, prefix, true);
+                } else {
+                    throw ex;
+                }
+            } catch (SdkException ex2) {
+                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex2, getMessageOrToString(ex));
+            }
+        } catch (SdkException ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex, getMessageOrToString(ex));
+        } finally {
+            if (s3Client != null) {
+                CleanupUtils.close(s3Client, null);
+            }
+        }
+
+        boolean isEmpty = useOldApi ? ((ListObjectsResponse) response).contents().isEmpty()
+                : ((ListObjectsV2Response) response).contents().isEmpty();
+        if (isEmpty && collector.shouldWarn()) {
+            Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+            collector.warn(warning);
+        }
+
+        // Returns 200 only in case the bucket exists, otherwise, throws an exception. However, to
+        // ensure coverage, check if the result is successful as well and not only catch exceptions
+        if (!response.sdkHttpResponse().isSuccessful()) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_CONTAINER_NOT_FOUND, container);
+        }
+        if (isDeltaTable(configuration)) {
+            try {
+                validateDeltaTableExists(configuration);
+            } catch (AlgebricksException e) {
+                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, e);
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Constants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Constants.java
index a62b346..126c868 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Constants.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Constants.java
@@ -28,6 +28,8 @@
     public static final String ACCESS_KEY_ID_FIELD_NAME = "accessKeyId";
     public static final String SECRET_ACCESS_KEY_FIELD_NAME = "secretAccessKey";
     public static final String SESSION_TOKEN_FIELD_NAME = "sessionToken";
+    public static final String ROLE_ARN_FIELD_NAME = "roleArn";
+    public static final String EXTERNAL_ID_FIELD_NAME = "externalId";
     public static final String SERVICE_END_POINT_FIELD_NAME = "serviceEndpoint";
 
     // AWS S3 specific error codes
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Utils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Utils.java
index bf0938b..d8dd478 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Utils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Utils.java
@@ -18,46 +18,18 @@
  */
 package org.apache.asterix.external.util.aws.s3;
 
-import static org.apache.asterix.common.exceptions.ErrorCode.INVALID_PARAM_VALUE_ALLOWED_VALUE;
-import static org.apache.asterix.common.exceptions.ErrorCode.PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT;
-import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_IF_PARAM_IS_PRESENT;
-import static org.apache.asterix.common.exceptions.ErrorCode.S3_REGION_NOT_SUPPORTED;
 import static org.apache.asterix.external.util.ExternalDataUtils.getPrefix;
-import static org.apache.asterix.external.util.ExternalDataUtils.isDeltaTable;
-import static org.apache.asterix.external.util.ExternalDataUtils.validateDeltaTableExists;
-import static org.apache.asterix.external.util.ExternalDataUtils.validateDeltaTableProperties;
-import static org.apache.asterix.external.util.ExternalDataUtils.validateIncludeExclude;
-import static org.apache.asterix.external.util.aws.s3.S3Constants.ACCESS_KEY_ID_FIELD_NAME;
-import static org.apache.asterix.external.util.aws.s3.S3Constants.ERROR_INTERNAL_ERROR;
 import static org.apache.asterix.external.util.aws.s3.S3Constants.ERROR_METHOD_NOT_IMPLEMENTED;
-import static org.apache.asterix.external.util.aws.s3.S3Constants.ERROR_SLOW_DOWN;
-import static org.apache.asterix.external.util.aws.s3.S3Constants.HADOOP_ACCESS_KEY_ID;
-import static org.apache.asterix.external.util.aws.s3.S3Constants.HADOOP_ANONYMOUS_ACCESS;
-import static org.apache.asterix.external.util.aws.s3.S3Constants.HADOOP_CREDENTIAL_PROVIDER_KEY;
-import static org.apache.asterix.external.util.aws.s3.S3Constants.HADOOP_PATH_STYLE_ACCESS;
-import static org.apache.asterix.external.util.aws.s3.S3Constants.HADOOP_S3_CONNECTION_POOL_SIZE;
-import static org.apache.asterix.external.util.aws.s3.S3Constants.HADOOP_S3_PROTOCOL;
-import static org.apache.asterix.external.util.aws.s3.S3Constants.HADOOP_SECRET_ACCESS_KEY;
-import static org.apache.asterix.external.util.aws.s3.S3Constants.HADOOP_SERVICE_END_POINT;
-import static org.apache.asterix.external.util.aws.s3.S3Constants.HADOOP_SESSION_TOKEN;
-import static org.apache.asterix.external.util.aws.s3.S3Constants.HADOOP_TEMP_ACCESS;
-import static org.apache.asterix.external.util.aws.s3.S3Constants.INSTANCE_PROFILE_FIELD_NAME;
-import static org.apache.asterix.external.util.aws.s3.S3Constants.REGION_FIELD_NAME;
-import static org.apache.asterix.external.util.aws.s3.S3Constants.SECRET_ACCESS_KEY_FIELD_NAME;
-import static org.apache.asterix.external.util.aws.s3.S3Constants.SERVICE_END_POINT_FIELD_NAME;
-import static org.apache.asterix.external.util.aws.s3.S3Constants.SESSION_TOKEN_FIELD_NAME;
 import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
 
-import java.net.URI;
-import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Optional;
 import java.util.function.BiPredicate;
 import java.util.regex.Matcher;
 
+import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.external.IExternalFilterEvaluator;
@@ -65,26 +37,13 @@
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.ExternalDataPrefix;
 import org.apache.asterix.external.util.ExternalDataUtils;
-import org.apache.asterix.external.util.HDFSUtils;
-import org.apache.hadoop.fs.s3a.Constants;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
-import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.exceptions.Warning;
 import org.apache.hyracks.api.util.CleanupUtils;
 
-import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
-import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
-import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
-import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
-import software.amazon.awssdk.auth.credentials.InstanceProfileCredentialsProvider;
-import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
 import software.amazon.awssdk.core.exception.SdkException;
-import software.amazon.awssdk.regions.Region;
 import software.amazon.awssdk.services.s3.S3Client;
-import software.amazon.awssdk.services.s3.S3ClientBuilder;
 import software.amazon.awssdk.services.s3.model.CommonPrefix;
 import software.amazon.awssdk.services.s3.model.ListObjectsRequest;
 import software.amazon.awssdk.services.s3.model.ListObjectsResponse;
@@ -100,257 +59,6 @@
         throw new AssertionError("do not instantiate");
     }
 
-    public static boolean isRetryableError(String errorCode) {
-        return errorCode.equals(ERROR_INTERNAL_ERROR) || errorCode.equals(ERROR_SLOW_DOWN);
-    }
-
-    /**
-     * Builds the S3 client using the provided configuration
-     *
-     * @param configuration properties
-     * @return S3 client
-     * @throws CompilationException CompilationException
-     */
-    public static S3Client buildAwsS3Client(Map<String, String> configuration) throws CompilationException {
-        // TODO(Hussain): Need to ensure that all required parameters are present in a previous step
-        String instanceProfile = configuration.get(INSTANCE_PROFILE_FIELD_NAME);
-        String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
-        String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
-        String sessionToken = configuration.get(SESSION_TOKEN_FIELD_NAME);
-        String regionId = configuration.get(REGION_FIELD_NAME);
-        String serviceEndpoint = configuration.get(SERVICE_END_POINT_FIELD_NAME);
-
-        S3ClientBuilder builder = S3Client.builder();
-
-        // Credentials
-        AwsCredentialsProvider credentialsProvider =
-                buildCredentialsProvider(instanceProfile, accessKeyId, secretAccessKey, sessionToken);
-
-        builder.credentialsProvider(credentialsProvider);
-
-        // Validate the region
-        List<Region> regions = S3Client.serviceMetadata().regions();
-        Optional<Region> selectedRegion = regions.stream().filter(region -> region.id().equals(regionId)).findFirst();
-
-        if (selectedRegion.isEmpty()) {
-            throw new CompilationException(S3_REGION_NOT_SUPPORTED, regionId);
-        }
-        builder.region(selectedRegion.get());
-
-        // Validate the service endpoint if present
-        if (serviceEndpoint != null) {
-            try {
-                URI uri = new URI(serviceEndpoint);
-                try {
-                    builder.endpointOverride(uri);
-                } catch (NullPointerException ex) {
-                    throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex, getMessageOrToString(ex));
-                }
-            } catch (URISyntaxException ex) {
-                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex,
-                        String.format("Invalid service endpoint %s", serviceEndpoint));
-            }
-        }
-
-        return builder.build();
-    }
-
-    public static AwsCredentialsProvider buildCredentialsProvider(String instanceProfile, String accessKeyId,
-            String secretAccessKey, String sessionToken) throws CompilationException {
-
-        // Credentials
-        AwsCredentialsProvider credentialsProvider;
-
-        // nothing provided, anonymous authentication
-        if (instanceProfile == null && accessKeyId == null && secretAccessKey == null && sessionToken == null) {
-            credentialsProvider = AnonymousCredentialsProvider.create();
-        } else if (instanceProfile != null) {
-
-            // only "true" value is allowed
-            if (!instanceProfile.equalsIgnoreCase("true")) {
-                throw new CompilationException(INVALID_PARAM_VALUE_ALLOWED_VALUE, INSTANCE_PROFILE_FIELD_NAME, "true");
-            }
-
-            // no other authentication parameters are allowed
-            if (accessKeyId != null) {
-                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
-                        INSTANCE_PROFILE_FIELD_NAME);
-            }
-            if (secretAccessKey != null) {
-                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, SECRET_ACCESS_KEY_FIELD_NAME,
-                        INSTANCE_PROFILE_FIELD_NAME);
-            }
-            if (sessionToken != null) {
-                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, SESSION_TOKEN_FIELD_NAME,
-                        INSTANCE_PROFILE_FIELD_NAME);
-            }
-            credentialsProvider = InstanceProfileCredentialsProvider.create();
-        } else if (accessKeyId != null || secretAccessKey != null) {
-            // accessKeyId authentication
-            if (accessKeyId == null) {
-                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
-                        SECRET_ACCESS_KEY_FIELD_NAME);
-            }
-            if (secretAccessKey == null) {
-                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, SECRET_ACCESS_KEY_FIELD_NAME,
-                        ACCESS_KEY_ID_FIELD_NAME);
-            }
-
-            // use session token if provided
-            if (sessionToken != null) {
-                credentialsProvider = StaticCredentialsProvider
-                        .create(AwsSessionCredentials.create(accessKeyId, secretAccessKey, sessionToken));
-            } else {
-                credentialsProvider =
-                        StaticCredentialsProvider.create(AwsBasicCredentials.create(accessKeyId, secretAccessKey));
-            }
-        } else {
-            // if only session token is provided, accessKeyId is required
-            throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
-                    SESSION_TOKEN_FIELD_NAME);
-        }
-        return credentialsProvider;
-    }
-
-    /**
-     * Builds the S3 client using the provided configuration
-     *
-     * @param configuration      properties
-     * @param numberOfPartitions number of partitions in the cluster
-     */
-    public static void configureAwsS3HdfsJobConf(JobConf conf, Map<String, String> configuration,
-            int numberOfPartitions) {
-        String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
-        String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
-        String sessionToken = configuration.get(SESSION_TOKEN_FIELD_NAME);
-        String serviceEndpoint = configuration.get(SERVICE_END_POINT_FIELD_NAME);
-
-        //Disable caching S3 FileSystem
-        HDFSUtils.disableHadoopFileSystemCache(conf, HADOOP_S3_PROTOCOL);
-
-        /*
-         * Authentication Methods:
-         * 1- Anonymous: no accessKeyId and no secretAccessKey
-         * 2- Temporary: has to provide accessKeyId, secretAccessKey and sessionToken
-         * 3- Private: has to provide accessKeyId and secretAccessKey
-         */
-        if (accessKeyId == null) {
-            //Tells hadoop-aws it is an anonymous access
-            conf.set(HADOOP_CREDENTIAL_PROVIDER_KEY, HADOOP_ANONYMOUS_ACCESS);
-        } else {
-            conf.set(HADOOP_ACCESS_KEY_ID, accessKeyId);
-            conf.set(HADOOP_SECRET_ACCESS_KEY, secretAccessKey);
-            if (sessionToken != null) {
-                conf.set(HADOOP_SESSION_TOKEN, sessionToken);
-                //Tells hadoop-aws it is a temporary access
-                conf.set(HADOOP_CREDENTIAL_PROVIDER_KEY, HADOOP_TEMP_ACCESS);
-            }
-        }
-
-        /*
-         * This is to allow S3 definition to have path-style form. Should always be true to match the current
-         * way we access files in S3
-         */
-        conf.set(HADOOP_PATH_STYLE_ACCESS, ExternalDataConstants.TRUE);
-
-        /*
-         * Set the size of S3 connection pool to be the number of partitions
-         */
-        conf.set(HADOOP_S3_CONNECTION_POOL_SIZE, String.valueOf(numberOfPartitions));
-
-        if (serviceEndpoint != null) {
-            // Validation of the URL should be done at hadoop-aws level
-            conf.set(HADOOP_SERVICE_END_POINT, serviceEndpoint);
-        } else {
-            //Region is ignored and buckets could be found by the central endpoint
-            conf.set(HADOOP_SERVICE_END_POINT, Constants.CENTRAL_ENDPOINT);
-        }
-    }
-
-    /**
-     * Validate external dataset properties
-     *
-     * @param configuration properties
-     * @throws CompilationException Compilation exception
-     */
-    public static void validateProperties(Map<String, String> configuration, SourceLocation srcLoc,
-            IWarningCollector collector) throws CompilationException {
-        if (isDeltaTable(configuration)) {
-            validateDeltaTableProperties(configuration);
-        }
-        // check if the format property is present
-        else if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
-            throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
-        }
-        // Both parameters should be passed, or neither should be passed (for anonymous/no auth)
-        String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
-        String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
-        if (accessKeyId == null || secretAccessKey == null) {
-            // If one is passed, the other is required
-            if (accessKeyId != null) {
-                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, SECRET_ACCESS_KEY_FIELD_NAME,
-                        ACCESS_KEY_ID_FIELD_NAME);
-            } else if (secretAccessKey != null) {
-                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
-                        SECRET_ACCESS_KEY_FIELD_NAME);
-            }
-        }
-
-        validateIncludeExclude(configuration);
-        try {
-            // TODO(htowaileb): maybe something better, this will check to ensure type is supported before creation
-            new ExternalDataPrefix(configuration);
-        } catch (AlgebricksException ex) {
-            throw new CompilationException(ErrorCode.FAILED_TO_CALCULATE_COMPUTED_FIELDS, ex);
-        }
-
-        // Check if the bucket is present
-        S3Client s3Client = buildAwsS3Client(configuration);
-        S3Response response;
-        boolean useOldApi = false;
-        String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-        String prefix = getPrefix(configuration);
-
-        try {
-            response = isBucketEmpty(s3Client, container, prefix, false);
-        } catch (S3Exception ex) {
-            // Method not implemented, try falling back to old API
-            try {
-                // For error code, see https://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html
-                if (ex.awsErrorDetails().errorCode().equals(ERROR_METHOD_NOT_IMPLEMENTED)) {
-                    useOldApi = true;
-                    response = isBucketEmpty(s3Client, container, prefix, true);
-                } else {
-                    throw ex;
-                }
-            } catch (SdkException ex2) {
-                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex2, getMessageOrToString(ex));
-            }
-        } catch (SdkException ex) {
-            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex, getMessageOrToString(ex));
-        } finally {
-            if (s3Client != null) {
-                CleanupUtils.close(s3Client, null);
-            }
-        }
-
-        boolean isEmpty = useOldApi ? ((ListObjectsResponse) response).contents().isEmpty()
-                : ((ListObjectsV2Response) response).contents().isEmpty();
-        if (isEmpty && collector.shouldWarn()) {
-            Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
-            collector.warn(warning);
-        }
-
-        // Returns 200 only in case the bucket exists, otherwise, throws an exception. However, to
-        // ensure coverage, check if the result is successful as well and not only catch exceptions
-        if (!response.sdkHttpResponse().isSuccessful()) {
-            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_CONTAINER_NOT_FOUND, container);
-        }
-        if (isDeltaTable(configuration)) {
-            validateDeltaTableExists(configuration);
-        }
-    }
-
     /**
      * Checks for a single object in the specified bucket to determine if the bucket is empty or not.
      *
@@ -360,7 +68,7 @@
      * @param useOldApi flag whether to use the old API or not
      * @return returns the S3 response
      */
-    private static S3Response isBucketEmpty(S3Client s3Client, String container, String prefix, boolean useOldApi) {
+    protected static S3Response isBucketEmpty(S3Client s3Client, String container, String prefix, boolean useOldApi) {
         S3Response response;
         if (useOldApi) {
             ListObjectsRequest.Builder listObjectsBuilder = ListObjectsRequest.builder();
@@ -380,14 +88,14 @@
      * @param configuration         properties
      * @param includeExcludeMatcher include/exclude matchers to apply
      */
-    public static List<S3Object> listS3Objects(Map<String, String> configuration,
+    public static List<S3Object> listS3Objects(IApplicationContext appCtx, Map<String, String> configuration,
             AbstractExternalInputStreamFactory.IncludeExcludeMatcher includeExcludeMatcher,
             IWarningCollector warningCollector, ExternalDataPrefix externalDataPrefix,
             IExternalFilterEvaluator evaluator) throws CompilationException, HyracksDataException {
         // Prepare to retrieve the objects
         List<S3Object> filesOnly;
         String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-        S3Client s3Client = buildAwsS3Client(configuration);
+        S3Client s3Client = S3AuthUtils.buildAwsS3Client(appCtx, configuration);
         String prefix = getPrefix(configuration);
 
         try {
@@ -530,10 +238,11 @@
         }
     }
 
-    public static Map<String, List<String>> S3ObjectsOfSingleDepth(Map<String, String> configuration, String container,
-            String prefix) throws CompilationException, HyracksDataException {
+    public static Map<String, List<String>> S3ObjectsOfSingleDepth(IApplicationContext appCtx,
+            Map<String, String> configuration, String container, String prefix)
+            throws CompilationException, HyracksDataException {
         // create s3 client
-        S3Client s3Client = buildAwsS3Client(configuration);
+        S3Client s3Client = S3AuthUtils.buildAwsS3Client(appCtx, configuration);
         // fetch all the s3 objects
         return listS3ObjectsOfSingleDepth(s3Client, container, prefix);
     }
@@ -546,7 +255,7 @@
      * @param prefix                definition prefix
      */
     private static Map<String, List<String>> listS3ObjectsOfSingleDepth(S3Client s3Client, String container,
-            String prefix) throws HyracksDataException {
+            String prefix) {
         Map<String, List<String>> allObjects = new HashMap<>();
         ListObjectsV2Iterable listObjectsInterable;
         ListObjectsV2Request.Builder listObjectsBuilder =
@@ -583,4 +292,10 @@
         allObjects.put("folders", folders);
         return allObjects;
     }
+
+    public static String getPath(Map<String, String> configuration) {
+        return S3Constants.HADOOP_S3_PROTOCOL + "://"
+                + configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME) + '/'
+                + configuration.get(ExternalDataConstants.DEFINITION_FIELD_NAME);
+    }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureUtils.java
index 4860ed1..fb594b9 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureUtils.java
@@ -123,7 +123,11 @@
         if (endpoint == null) {
             throw new CompilationException(PARAMETERS_REQUIRED, ENDPOINT_FIELD_NAME);
         }
-        builder.endpoint(endpoint);
+        try {
+            builder.endpoint(endpoint);
+        } catch (Exception ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex, getMessageOrToString(ex));
+        }
 
         // Shared Key
         if (accountName != null || accountKey != null) {
@@ -276,7 +280,11 @@
         if (endpoint == null) {
             throw new CompilationException(PARAMETERS_REQUIRED, ENDPOINT_FIELD_NAME);
         }
-        builder.endpoint(endpoint);
+        try {
+            builder.endpoint(endpoint);
+        } catch (Exception ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex, getMessageOrToString(ex));
+        }
 
         // Shared Key
         if (accountName != null || accountKey != null) {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSConstants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSConstants.java
index 2818043..2613f34 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSConstants.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSConstants.java
@@ -26,6 +26,7 @@
     public static final String APPLICATION_DEFAULT_CREDENTIALS_FIELD_NAME = "applicationDefaultCredentials";
     public static final String JSON_CREDENTIALS_FIELD_NAME = "jsonCredentials";
     public static final String ENDPOINT_FIELD_NAME = "endpoint";
+    public static final String STORAGE_PREFIX = "prefix";
 
     /*
      * Hadoop internal configuration
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSUtils.java
index 74a664d..bfd35fc 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSUtils.java
@@ -22,6 +22,8 @@
 import static org.apache.asterix.common.exceptions.ErrorCode.INVALID_PARAM_VALUE_ALLOWED_VALUE;
 import static org.apache.asterix.common.exceptions.ErrorCode.PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT;
 import static org.apache.asterix.external.util.ExternalDataUtils.getPrefix;
+import static org.apache.asterix.external.util.ExternalDataUtils.isDeltaTable;
+import static org.apache.asterix.external.util.ExternalDataUtils.validateDeltaTableProperties;
 import static org.apache.asterix.external.util.ExternalDataUtils.validateIncludeExclude;
 import static org.apache.asterix.external.util.google.gcs.GCSConstants.APPLICATION_DEFAULT_CREDENTIALS_FIELD_NAME;
 import static org.apache.asterix.external.util.google.gcs.GCSConstants.ENDPOINT_FIELD_NAME;
@@ -140,9 +142,11 @@
      */
     public static void validateProperties(Map<String, String> configuration, SourceLocation srcLoc,
             IWarningCollector collector) throws CompilationException {
-
+        if (isDeltaTable(configuration)) {
+            validateDeltaTableProperties(configuration);
+        }
         // check if the format property is present
-        if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
+        else if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
             throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
         }
 
@@ -224,6 +228,11 @@
      * @param configuration      properties
      * @param numberOfPartitions number of partitions in the cluster
      */
+    public static void configureHdfsJobConf(JobConf conf, Map<String, String> configuration)
+            throws AlgebricksException {
+        configureHdfsJobConf(conf, configuration, 0);
+    }
+
     public static void configureHdfsJobConf(JobConf conf, Map<String, String> configuration, int numberOfPartitions)
             throws AlgebricksException {
         String jsonCredentials = configuration.get(JSON_CREDENTIALS_FIELD_NAME);
@@ -267,4 +276,10 @@
             conf.set(HADOOP_ENDPOINT, endpoint);
         }
     }
+
+    public static String getPath(Map<String, String> configuration) {
+        return GCSConstants.HADOOP_GCS_PROTOCOL + "://"
+                + configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME) + '/'
+                + configuration.get(ExternalDataConstants.DEFINITION_FIELD_NAME);
+    }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/HDFSExternalFileWriter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/HDFSExternalFileWriter.java
new file mode 100644
index 0000000..f06b8e9
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/HDFSExternalFileWriter.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.writer;
+
+import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.external.util.HDFSUtils;
+import org.apache.asterix.runtime.writer.IExternalFileWriter;
+import org.apache.asterix.runtime.writer.IExternalPrinter;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.parquet.hadoop.util.HiddenFileFilter;
+
+public class HDFSExternalFileWriter implements IExternalFileWriter {
+
+    private final IExternalPrinter printer;
+    private final FileSystem fs;
+    private final boolean partitionedPath;
+    private final SourceLocation pathSourceLocation;
+    private FSDataOutputStream outputStream = null;
+    private final List<Path> paths = new ArrayList<>();
+
+    HDFSExternalFileWriter(IExternalPrinter printer, FileSystem fs, boolean partitionedPath,
+            SourceLocation pathSourceLocation) {
+        this.printer = printer;
+        this.fs = fs;
+        this.partitionedPath = partitionedPath;
+        this.pathSourceLocation = pathSourceLocation;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        printer.open();
+    }
+
+    @Override
+    public void validate(String directory) throws HyracksDataException {
+        if (partitionedPath) {
+            directory = HDFSUtils.updateRootPath(directory, true);
+            Path dirPath = new Path(directory);
+            try {
+                if (fs.exists(dirPath)) {
+                    FileStatus fileStatus = fs.getFileStatus(dirPath);
+                    if (fileStatus.isFile()) {
+                        throw new RuntimeDataException(ErrorCode.DIRECTORY_IS_NOT_EMPTY, pathSourceLocation, directory);
+                    }
+                    if (fileStatus.isDirectory()) {
+                        FileStatus[] fileStatuses = fs.listStatus(dirPath, HiddenFileFilter.INSTANCE);
+                        if (fileStatuses.length != 0) {
+                            throw new RuntimeDataException(ErrorCode.DIRECTORY_IS_NOT_EMPTY, pathSourceLocation,
+                                    directory);
+                        }
+                    }
+                }
+            } catch (IOException ex) {
+                throw RuntimeDataException.create(ErrorCode.EXTERNAL_SINK_ERROR, ex, getMessageOrToString(ex));
+            }
+        }
+    }
+
+    @Override
+    public boolean newFile(String directory, String fileName) throws HyracksDataException {
+        directory = HDFSUtils.updateRootPath(directory, true);
+        Path path = new Path(directory, "." + fileName);
+        try {
+            outputStream = fs.create(path, false);
+            paths.add(path);
+            printer.newStream(outputStream);
+        } catch (FileAlreadyExistsException e) {
+            return false;
+        } catch (IOException ex) {
+            throw RuntimeDataException.create(ErrorCode.EXTERNAL_SINK_ERROR, ex, getMessageOrToString(ex));
+        }
+        return true;
+    }
+
+    @Override
+    public void write(IValueReference value) throws HyracksDataException {
+        printer.print(value);
+    }
+
+    @Override
+    public void abort() throws HyracksDataException {
+        try {
+            printer.close();
+            for (Path path : paths) {
+                fs.delete(path, false);
+            }
+        } catch (IOException ex) {
+            throw RuntimeDataException.create(ErrorCode.EXTERNAL_SINK_ERROR, ex, getMessageOrToString(ex));
+        }
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        printer.close();
+        try {
+            for (Path path : paths) {
+                fs.rename(path, new Path(path.getParent(), path.getName().substring(1)));
+            }
+        } catch (IOException ex) {
+            throw RuntimeDataException.create(ErrorCode.EXTERNAL_SINK_ERROR, ex, getMessageOrToString(ex));
+        }
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/HDFSExternalFileWriterFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/HDFSExternalFileWriterFactory.java
new file mode 100644
index 0000000..7bb07bc
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/HDFSExternalFileWriterFactory.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.writer;
+
+import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.Map;
+import java.util.UUID;
+
+import org.apache.asterix.common.api.IApplicationContext;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.HDFSUtils;
+import org.apache.asterix.runtime.writer.ExternalFileWriterConfiguration;
+import org.apache.asterix.runtime.writer.IExternalFileWriter;
+import org.apache.asterix.runtime.writer.IExternalFileWriterFactory;
+import org.apache.asterix.runtime.writer.IExternalFileWriterFactoryProvider;
+import org.apache.asterix.runtime.writer.IExternalPrinter;
+import org.apache.asterix.runtime.writer.IExternalPrinterFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.util.ExceptionUtils;
+
+public class HDFSExternalFileWriterFactory implements IExternalFileWriterFactory {
+    private static final long serialVersionUID = 1L;
+    private static final char SEPARATOR = '/';
+    public static final IExternalFileWriterFactoryProvider PROVIDER = new IExternalFileWriterFactoryProvider() {
+        @Override
+        public IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration) {
+            return new HDFSExternalFileWriterFactory(configuration);
+        }
+
+        @Override
+        public char getSeparator() {
+            return SEPARATOR;
+        }
+    };
+
+    private final Map<String, String> configuration;
+    private final String staticPath;
+    private final SourceLocation pathSourceLocation;
+    private transient Credentials credentials;
+    private byte[] serializedCredentials;
+    private transient FileSystem fs;
+
+    private HDFSExternalFileWriterFactory(ExternalFileWriterConfiguration externalConfig) {
+        configuration = externalConfig.getConfiguration();
+        staticPath = HDFSUtils.updateRootPath(externalConfig.getStaticPath(), false);
+        pathSourceLocation = externalConfig.getPathSourceLocation();
+    }
+
+    private FileSystem createFileSystem(Configuration conf) throws CompilationException, HyracksDataException {
+        try {
+            if (credentials != null) {
+                UserGroupInformation ugi = UserGroupInformation.createRemoteUser(UUID.randomUUID().toString());
+                ugi.addCredentials(credentials);
+                return ugi.doAs((PrivilegedExceptionAction<FileSystem>) () -> FileSystem.get(conf));
+            }
+            return FileSystem.get(conf);
+        } catch (InterruptedException ex) {
+            throw HyracksDataException.create(ex);
+        } catch (IOException ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SINK_ERROR, ex, getMessageOrToString(ex));
+        }
+    }
+
+    private void buildFileSystem() throws HyracksDataException {
+        try {
+            synchronized (this) {
+                if (credentials == null && serializedCredentials != null) {
+                    credentials = new Credentials();
+                    HDFSUtils.deserialize(serializedCredentials, credentials);
+                }
+                if (fs == null) {
+                    Configuration conf = HDFSUtils.configureHDFSwrite(configuration);
+                    fs = createFileSystem(conf);
+                }
+            }
+        } catch (IOException | CompilationException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    @Override
+    public IExternalFileWriter createWriter(IHyracksTaskContext context, IExternalPrinterFactory printerFactory)
+            throws HyracksDataException {
+        buildFileSystem();
+        IExternalPrinter printer = printerFactory.createPrinter();
+        return new HDFSExternalFileWriter(printer, fs, staticPath == null, pathSourceLocation);
+    }
+
+    @Override
+    public char getSeparator() {
+        return SEPARATOR;
+    }
+
+    @Override
+    public void validate(IApplicationContext appCtx) throws AlgebricksException {
+        Configuration conf = HDFSUtils.configureHDFSwrite(configuration);
+        credentials = HDFSUtils.configureHadoopAuthentication(configuration, conf);
+        try {
+            if (credentials != null) {
+                serializedCredentials = HDFSUtils.serialize(credentials);
+            }
+            try (FileSystem testFs = createFileSystem(conf)) {
+                doValidate(testFs);
+            }
+        } catch (IOException ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SINK_ERROR, ExceptionUtils.getMessageOrToString(ex));
+        }
+    }
+
+    private void doValidate(FileSystem testFs) throws IOException, AlgebricksException {
+        if (staticPath != null) {
+            Path dirPath = new Path(staticPath);
+            if (testFs.exists(dirPath)) {
+                FileStatus fileStatus = testFs.getFileStatus(dirPath);
+                if (fileStatus.isFile()) {
+                    throw new CompilationException(ErrorCode.DIRECTORY_IS_NOT_EMPTY, pathSourceLocation, staticPath);
+                }
+                if (fileStatus.isDirectory()) {
+                    FileStatus[] fileStatuses = testFs.listStatus(dirPath);
+                    if (fileStatuses.length != 0) {
+                        throw new CompilationException(ErrorCode.DIRECTORY_IS_NOT_EMPTY, pathSourceLocation,
+                                staticPath);
+                    }
+                }
+            }
+            checkDirectoryWritePermission(testFs);
+        }
+    }
+
+    private void checkDirectoryWritePermission(FileSystem fs) throws AlgebricksException {
+        if (!Boolean.parseBoolean(configuration.getOrDefault(ExternalDataConstants.KEY_VALIDATE_WRITE_PERMISSION,
+                Boolean.TRUE.toString()))) {
+            return;
+        }
+        Path path = new Path(staticPath, "testFile");
+        try {
+            try (FSDataOutputStream outputStream = fs.create(path)) {
+                fs.deleteOnExit(path);
+                outputStream.write(0);
+            }
+        } catch (IOException ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SINK_ERROR, ex, getMessageOrToString(ex));
+        }
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/LocalFSExternalFileWriterFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/LocalFSExternalFileWriterFactory.java
index bdefaa8..b1d3a95 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/LocalFSExternalFileWriterFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/LocalFSExternalFileWriterFactory.java
@@ -20,6 +20,7 @@
 
 import java.io.File;
 
+import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
@@ -80,7 +81,7 @@
     }
 
     @Override
-    public void validate() throws AlgebricksException {
+    public void validate(IApplicationContext appCtx) throws AlgebricksException {
         // A special case validation for a single node cluster
         if (singleNodeCluster && staticPath != null) {
             if (isNonEmptyDirectory(new File(staticPath))) {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/CsvExternalFilePrinter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/CsvExternalFilePrinter.java
new file mode 100644
index 0000000..b5299ad
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/CsvExternalFilePrinter.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.external.writer.printer;
+
+import org.apache.asterix.external.writer.compressor.IExternalFileCompressStreamFactory;
+import org.apache.hyracks.algebricks.data.IPrinter;
+
+public class CsvExternalFilePrinter extends AbstractTextualExternalPrinter {
+    CsvExternalFilePrinter(IPrinter printer, IExternalFileCompressStreamFactory compressStreamFactory) {
+        super(printer, compressStreamFactory);
+    }
+
+    @Override
+    void afterPrint() {
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/CsvExternalFilePrinterFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/CsvExternalFilePrinterFactory.java
new file mode 100644
index 0000000..0ed1498
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/CsvExternalFilePrinterFactory.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.external.writer.printer;
+
+import org.apache.asterix.external.writer.compressor.IExternalFileCompressStreamFactory;
+import org.apache.asterix.runtime.writer.IExternalPrinter;
+import org.apache.asterix.runtime.writer.IExternalPrinterFactory;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+public class CsvExternalFilePrinterFactory implements IExternalPrinterFactory {
+    private static final long serialVersionUID = 8971234908711234L;
+    protected final IPrinterFactory printerFactory;
+    private final IExternalFileCompressStreamFactory compressStreamFactory;
+
+    public CsvExternalFilePrinterFactory(IPrinterFactory printerFactory,
+            IExternalFileCompressStreamFactory compressStreamFactory) {
+        this.printerFactory = printerFactory;
+        this.compressStreamFactory = compressStreamFactory;
+    }
+
+    @Override
+    public IExternalPrinter createPrinter() {
+        return new CsvExternalFilePrinter(printerFactory.createPrinter(), compressStreamFactory);
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/ParquetExternalFilePrinter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/ParquetExternalFilePrinter.java
new file mode 100644
index 0000000..ba7a1ee
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/ParquetExternalFilePrinter.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.external.writer.printer;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+import org.apache.asterix.external.input.record.reader.hdfs.parquet.AsterixParquetRuntimeException;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.writer.printer.parquet.AsterixParquetWriter;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.writer.IExternalPrinter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.MessageTypeParser;
+
+public class ParquetExternalFilePrinter implements IExternalPrinter {
+    private final IAType typeInfo;
+    private final CompressionCodecName compressionCodecName;
+    private MessageType schema;
+    private ParquetOutputFile parquetOutputFile;
+    private String parquetSchemaString;
+    private ParquetWriter<IValueReference> writer;
+    private final long rowGroupSize;
+    private final int pageSize;
+    private final ParquetProperties.WriterVersion writerVersion;
+
+    public ParquetExternalFilePrinter(CompressionCodecName compressionCodecName, String parquetSchemaString,
+            IAType typeInfo, long rowGroupSize, int pageSize, ParquetProperties.WriterVersion writerVersion) {
+        this.compressionCodecName = compressionCodecName;
+        this.parquetSchemaString = parquetSchemaString;
+        this.typeInfo = typeInfo;
+        this.rowGroupSize = rowGroupSize;
+        this.pageSize = pageSize;
+        this.writerVersion = writerVersion;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        schema = MessageTypeParser.parseMessageType(parquetSchemaString);
+    }
+
+    @Override
+    public void newStream(OutputStream outputStream) throws HyracksDataException {
+        if (parquetOutputFile != null) {
+            close();
+        }
+        parquetOutputFile = new ParquetOutputFile(outputStream);
+        Configuration conf = new Configuration();
+
+        try {
+            writer = AsterixParquetWriter.builder(parquetOutputFile).withCompressionCodec(compressionCodecName)
+                    .withType(schema).withTypeInfo(typeInfo).withRowGroupSize(rowGroupSize).withPageSize(pageSize)
+                    .withDictionaryPageSize(ExternalDataConstants.PARQUET_DICTIONARY_PAGE_SIZE)
+                    .enableDictionaryEncoding().withValidation(false).withWriterVersion(writerVersion).withConf(conf)
+                    .build();
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+
+    }
+
+    @Override
+    public void print(IValueReference value) throws HyracksDataException {
+        try {
+            this.writer.write(value);
+        } catch (AsterixParquetRuntimeException e) {
+            throw e.getHyracksDataException();
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        if (this.writer != null) {
+            try {
+                this.writer.close();
+            } catch (IOException e) {
+                throw HyracksDataException.create(e);
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/ParquetExternalFilePrinterFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/ParquetExternalFilePrinterFactory.java
new file mode 100644
index 0000000..b6ad34e
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/ParquetExternalFilePrinterFactory.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.writer.printer;
+
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.writer.IExternalPrinter;
+import org.apache.asterix.runtime.writer.IExternalPrinterFactory;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+public class ParquetExternalFilePrinterFactory implements IExternalPrinterFactory {
+    private static final long serialVersionUID = 8971234908711235L;
+    private String parquetSchemaString;
+    private final IAType typeInfo;
+    private final CompressionCodecName compressionCodecName;
+    private final long rowGroupSize;
+    private final int pageSize;
+    private final ParquetProperties.WriterVersion writerVersion;
+
+    public ParquetExternalFilePrinterFactory(CompressionCodecName compressionCodecName, String parquetSchemaString,
+            IAType typeInfo, long rowGroupSize, int pageSize, ParquetProperties.WriterVersion writerVersion) {
+        this.compressionCodecName = compressionCodecName;
+        this.parquetSchemaString = parquetSchemaString;
+        this.typeInfo = typeInfo;
+        this.rowGroupSize = rowGroupSize;
+        this.pageSize = pageSize;
+        this.writerVersion = writerVersion;
+    }
+
+    public ParquetExternalFilePrinterFactory(CompressionCodecName compressionCodecName, IAType typeInfo,
+            long rowGroupSize, int pageSize, ParquetProperties.WriterVersion writerVersion) {
+        this.compressionCodecName = compressionCodecName;
+        this.typeInfo = typeInfo;
+        this.rowGroupSize = rowGroupSize;
+        this.pageSize = pageSize;
+        this.writerVersion = writerVersion;
+    }
+
+    public void setParquetSchemaString(String parquetSchemaString) {
+        this.parquetSchemaString = parquetSchemaString;
+    }
+
+    @Override
+    public IExternalPrinter createPrinter() {
+        return new ParquetExternalFilePrinter(compressionCodecName, parquetSchemaString, typeInfo, rowGroupSize,
+                pageSize, writerVersion);
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/ParquetExternalFilePrinterFactoryProvider.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/ParquetExternalFilePrinterFactoryProvider.java
new file mode 100644
index 0000000..bd70853
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/ParquetExternalFilePrinterFactoryProvider.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.writer.printer;
+
+import java.io.Serializable;
+
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.writer.IExternalPrinterFactory;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+public class ParquetExternalFilePrinterFactoryProvider implements Serializable {
+    private static final long serialVersionUID = 8971234908711237L;
+    private final IAType typeInfo;
+    private final CompressionCodecName compressionCodecName;
+    private final long rowGroupSize;
+    private final int pageSize;
+    private final ParquetProperties.WriterVersion writerVersion;
+
+    public ParquetExternalFilePrinterFactoryProvider(CompressionCodecName compressionCodecName, IAType typeInfo,
+            long rowGroupSize, int pageSize, ParquetProperties.WriterVersion writerVersion) {
+        this.compressionCodecName = compressionCodecName;
+        this.typeInfo = typeInfo;
+        this.rowGroupSize = rowGroupSize;
+        this.pageSize = pageSize;
+        this.writerVersion = writerVersion;
+    }
+
+    public IExternalPrinterFactory createPrinterFactory() {
+        return new ParquetExternalFilePrinterFactory(compressionCodecName, typeInfo, rowGroupSize, pageSize,
+                writerVersion);
+    }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/ParquetOutputFile.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/ParquetOutputFile.java
new file mode 100644
index 0000000..5db600f
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/ParquetOutputFile.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.external.writer.printer;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.parquet.hadoop.util.HadoopStreams;
+import org.apache.parquet.io.OutputFile;
+import org.apache.parquet.io.PositionOutputStream;
+
+public class ParquetOutputFile implements OutputFile {
+    private final FSDataOutputStream fs;
+
+    /*
+     This class wraps OutputStream as a file that Parquet SDK supports writing to.
+     By default, this assumes output stream doesn't support block size which distributed file systems use.
+     Hadoop File System Library use this as a default block size
+     Ref : https://github.com/apache/hadoop/blob/74ff00705cf67911f1ff8320c6c97354350d6952/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java#L2756
+     */
+    private static final long DEFAULT_BLOCK_SIZE = 33554432L;
+
+    public ParquetOutputFile(OutputStream os) {
+        this.fs = new FSDataOutputStream(os, new FileSystem.Statistics("test"));
+    }
+
+    @Override
+    public PositionOutputStream create(long blockSizeHint) throws IOException {
+        return HadoopStreams.wrap(fs);
+    }
+
+    @Override
+    public PositionOutputStream createOrOverwrite(long blockSizeHint) throws IOException {
+        return HadoopStreams.wrap(fs);
+    }
+
+    @Override
+    public boolean supportsBlockSize() {
+        return false;
+    }
+
+    @Override
+    public long defaultBlockSize() {
+        return DEFAULT_BLOCK_SIZE;
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/AsterixParquetTypeMap.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/AsterixParquetTypeMap.java
new file mode 100644
index 0000000..0dcdb3a
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/AsterixParquetTypeMap.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.writer.printer.parquet;
+
+import java.util.Map;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.parquet.schema.LogicalTypeAnnotation;
+import org.apache.parquet.schema.PrimitiveType;
+
+public class AsterixParquetTypeMap {
+
+    public static final Map<ATypeTag, PrimitiveType.PrimitiveTypeName> PRIMITIVE_TYPE_NAME_MAP =
+            Map.ofEntries(Map.entry(ATypeTag.BOOLEAN, PrimitiveType.PrimitiveTypeName.BOOLEAN),
+                    Map.entry(ATypeTag.STRING, PrimitiveType.PrimitiveTypeName.BINARY),
+                    Map.entry(ATypeTag.TINYINT, PrimitiveType.PrimitiveTypeName.INT32),
+                    Map.entry(ATypeTag.SMALLINT, PrimitiveType.PrimitiveTypeName.INT32),
+                    Map.entry(ATypeTag.INTEGER, PrimitiveType.PrimitiveTypeName.INT32),
+                    Map.entry(ATypeTag.BIGINT, PrimitiveType.PrimitiveTypeName.INT64),
+                    Map.entry(ATypeTag.FLOAT, PrimitiveType.PrimitiveTypeName.FLOAT),
+                    Map.entry(ATypeTag.DOUBLE, PrimitiveType.PrimitiveTypeName.DOUBLE),
+                    Map.entry(ATypeTag.DATE, PrimitiveType.PrimitiveTypeName.INT32),
+                    Map.entry(ATypeTag.TIME, PrimitiveType.PrimitiveTypeName.INT32),
+                    Map.entry(ATypeTag.DATETIME, PrimitiveType.PrimitiveTypeName.INT64));
+
+    public static final Map<ATypeTag, LogicalTypeAnnotation> LOGICAL_TYPE_ANNOTATION_MAP =
+            Map.ofEntries(Map.entry(ATypeTag.STRING, LogicalTypeAnnotation.stringType()),
+                    Map.entry(ATypeTag.DATE, LogicalTypeAnnotation.dateType()),
+                    Map.entry(ATypeTag.TIME,
+                            LogicalTypeAnnotation.timeType(true, LogicalTypeAnnotation.TimeUnit.MILLIS)),
+                    Map.entry(ATypeTag.DATETIME,
+                            LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MILLIS)));
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/AsterixParquetWriter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/AsterixParquetWriter.java
new file mode 100644
index 0000000..edeab1f
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/AsterixParquetWriter.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.writer.printer.parquet;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.om.types.IAType;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.api.WriteSupport;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.io.OutputFile;
+import org.apache.parquet.schema.MessageType;
+
+public class AsterixParquetWriter extends ParquetWriter<IValueReference> {
+    public static Builder builder(Path file) {
+        return new Builder(file);
+    }
+
+    public static Builder builder(OutputFile file) {
+        return new Builder(file);
+    }
+
+    AsterixParquetWriter(Path file, WriteSupport<IValueReference> writeSupport,
+            CompressionCodecName compressionCodecName, int blockSize, int pageSize, boolean enableDictionary,
+            boolean enableValidation, ParquetProperties.WriterVersion writerVersion, Configuration conf)
+            throws IOException {
+        super(file, writeSupport, compressionCodecName, blockSize, pageSize, pageSize, enableDictionary,
+                enableValidation, writerVersion, conf);
+    }
+
+    public static class Builder extends ParquetWriter.Builder<IValueReference, Builder> {
+        private MessageType type;
+        private IAType typeInfo;
+        private Map<String, String> extraMetaData;
+
+        private Builder(Path file) {
+            super(file);
+            this.type = null;
+            this.extraMetaData = new HashMap();
+        }
+
+        private Builder(OutputFile file) {
+            super(file);
+            this.type = null;
+            this.extraMetaData = new HashMap();
+        }
+
+        public Builder withType(MessageType type) {
+            this.type = type;
+            return this;
+        }
+
+        public Builder withTypeInfo(IAType typeInfo) {
+            this.typeInfo = typeInfo;
+            return this;
+        }
+
+        public Builder withExtraMetaData(Map<String, String> extraMetaData) {
+            this.extraMetaData = extraMetaData;
+            return this;
+        }
+
+        protected Builder self() {
+            return this;
+        }
+
+        protected WriteSupport<IValueReference> getWriteSupport(Configuration conf) {
+            return new ObjectWriteSupport(this.type, this.typeInfo, this.extraMetaData);
+        }
+    }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/FieldNamesDictionary.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/FieldNamesDictionary.java
new file mode 100644
index 0000000..7058bf6
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/FieldNamesDictionary.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.writer.printer.parquet;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.om.dictionary.FieldNamesTrieDictionary;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.util.string.UTF8StringUtil;
+
+public class FieldNamesDictionary {
+    private final FieldNamesTrieDictionary trie;
+    private final List<String> fieldNames;
+    private final StringBuilder builder;
+
+    public FieldNamesDictionary() {
+        trie = new FieldNamesTrieDictionary();
+        fieldNames = new ArrayList<>();
+        builder = new StringBuilder();
+    }
+
+    public String getOrCreateFieldNameIndex(IValueReference pointable) throws HyracksDataException {
+        int index = trie.getOrCreateFieldNameIndex(pointable);
+        if (index < fieldNames.size()) {
+            return fieldNames.get(index);
+        }
+
+        builder.setLength(0);
+        String fieldName = UTF8StringUtil.toString(pointable.getByteArray(), pointable.getStartOffset(), builder);
+        fieldNames.add(fieldName);
+        return fieldName;
+    }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/ISchemaChecker.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/ISchemaChecker.java
new file mode 100644
index 0000000..99b9736
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/ISchemaChecker.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.writer.printer.parquet;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+public interface ISchemaChecker {
+    enum SchemaComparisonType {
+        EQUIVALENT,
+        GROWING,
+        CONFLICTING
+    }
+
+    static SchemaComparisonType max(ISchemaChecker.SchemaComparisonType a, ISchemaChecker.SchemaComparisonType b) {
+        return a.compareTo(b) > 0 ? a : b;
+    }
+
+    SchemaComparisonType checkSchema(ParquetSchemaTree.SchemaNode schemaNode, IValueReference iValueReference)
+            throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/ObjectWriteSupport.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/ObjectWriteSupport.java
new file mode 100644
index 0000000..512b523
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/ObjectWriteSupport.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.writer.printer.parquet;
+
+import java.util.Map;
+
+import org.apache.asterix.external.input.record.reader.hdfs.parquet.AsterixParquetRuntimeException;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.parquet.hadoop.api.WriteSupport;
+import org.apache.parquet.io.api.RecordConsumer;
+import org.apache.parquet.schema.MessageType;
+
+public class ObjectWriteSupport extends WriteSupport<IValueReference> {
+    private MessageType schema;
+
+    private RecordConsumer recordConsumer;
+    private Map<String, String> extraMetaData;
+    ParquetRecordLazyVisitor parquetRecordLazyVisitor;
+
+    public ObjectWriteSupport(MessageType schema, IAType typeInfo, Map<String, String> extraMetaData) {
+        this.schema = schema;
+        this.extraMetaData = extraMetaData;
+        parquetRecordLazyVisitor = new ParquetRecordLazyVisitor(schema, typeInfo);
+    }
+
+    public String getName() {
+        return "asterix";
+    }
+
+    public WriteSupport.WriteContext init(Configuration configuration) {
+        return new WriteSupport.WriteContext(this.schema, this.extraMetaData);
+    }
+
+    public void prepareForWrite(RecordConsumer recordConsumer) {
+        this.recordConsumer = recordConsumer;
+    }
+
+    @Override
+    public void write(IValueReference valueReference) {
+        try {
+            parquetRecordLazyVisitor.consumeRecord(valueReference, recordConsumer);
+        } catch (HyracksDataException e) {
+            throw new AsterixParquetRuntimeException(e);
+        }
+    }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/ParquetRecordLazyVisitor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/ParquetRecordLazyVisitor.java
new file mode 100644
index 0000000..373bfe4
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/ParquetRecordLazyVisitor.java
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.writer.printer.parquet;
+
+import static org.apache.asterix.external.writer.printer.parquet.ParquetValueWriter.ELEMENT_FIELD;
+import static org.apache.asterix.external.writer.printer.parquet.ParquetValueWriter.GROUP_TYPE_ERROR_FIELD;
+import static org.apache.asterix.external.writer.printer.parquet.ParquetValueWriter.LIST_FIELD;
+import static org.apache.asterix.external.writer.printer.parquet.ParquetValueWriter.PRIMITIVE_TYPE_ERROR_FIELD;
+
+import org.apache.asterix.om.lazy.AbstractLazyVisitablePointable;
+import org.apache.asterix.om.lazy.AbstractListLazyVisitablePointable;
+import org.apache.asterix.om.lazy.FlatLazyVisitablePointable;
+import org.apache.asterix.om.lazy.ILazyVisitablePointableVisitor;
+import org.apache.asterix.om.lazy.RecordLazyVisitablePointable;
+import org.apache.asterix.om.lazy.TypedRecordLazyVisitablePointable;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.parquet.io.api.RecordConsumer;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.Type;
+
+public class ParquetRecordLazyVisitor implements ILazyVisitablePointableVisitor<Void, Type> {
+
+    private final MessageType schema;
+    private final RecordLazyVisitablePointable rec;
+    private RecordConsumer recordConsumer;
+    private final FieldNamesDictionary fieldNamesDictionary;
+
+    private final ParquetValueWriter parquetValueWriter;
+
+    public ParquetRecordLazyVisitor(MessageType schema, IAType typeInfo) {
+        this.schema = schema;
+        if (typeInfo.getTypeTag() == ATypeTag.OBJECT) {
+            this.rec = new TypedRecordLazyVisitablePointable((ARecordType) typeInfo);
+        } else if (typeInfo.getTypeTag() == ATypeTag.ANY) {
+            this.rec = new RecordLazyVisitablePointable(true);
+        } else {
+            throw new RuntimeException("Type Unsupported for parquet printing");
+        }
+        this.fieldNamesDictionary = new FieldNamesDictionary();
+        this.parquetValueWriter = new ParquetValueWriter();
+    }
+
+    public MessageType getSchema() {
+        return schema;
+    }
+
+    @Override
+    public Void visit(RecordLazyVisitablePointable pointable, Type type) throws HyracksDataException {
+
+        if (type.isPrimitive()) {
+            throw new HyracksDataException(ErrorCode.RESULT_DOES_NOT_FOLLOW_SCHEMA, GROUP_TYPE_ERROR_FIELD,
+                    PRIMITIVE_TYPE_ERROR_FIELD, type.getName());
+        }
+        GroupType groupType = type.asGroupType();
+        recordConsumer.startGroup();
+
+        for (int i = 0; i < pointable.getNumberOfChildren(); i++) {
+            pointable.nextChild();
+            AbstractLazyVisitablePointable child = pointable.getChildVisitablePointable();
+            String columnName = fieldNamesDictionary.getOrCreateFieldNameIndex(pointable.getFieldName());
+
+            if (!groupType.containsField(columnName)) {
+                throw new HyracksDataException(ErrorCode.EXTRA_FIELD_IN_RESULT_NOT_FOUND_IN_SCHEMA, columnName,
+                        groupType.getName());
+            }
+            recordConsumer.startField(columnName, groupType.getFieldIndex(columnName));
+            child.accept(this, groupType.getType(columnName));
+            recordConsumer.endField(columnName, groupType.getFieldIndex(columnName));
+        }
+        recordConsumer.endGroup();
+        return null;
+    }
+
+    @Override
+    public Void visit(AbstractListLazyVisitablePointable pointable, Type type) throws HyracksDataException {
+
+        if (type.isPrimitive()) {
+            throw new HyracksDataException(ErrorCode.RESULT_DOES_NOT_FOLLOW_SCHEMA, GROUP_TYPE_ERROR_FIELD,
+                    PRIMITIVE_TYPE_ERROR_FIELD, type.getName());
+        }
+        GroupType groupType = type.asGroupType();
+
+        if (!groupType.containsField(LIST_FIELD)) {
+            throw new HyracksDataException(ErrorCode.EXTRA_FIELD_IN_RESULT_NOT_FOUND_IN_SCHEMA, LIST_FIELD,
+                    groupType.getName());
+        }
+
+        if (groupType.getType(LIST_FIELD).isPrimitive()) {
+            throw new HyracksDataException(ErrorCode.RESULT_DOES_NOT_FOLLOW_SCHEMA, GROUP_TYPE_ERROR_FIELD,
+                    PRIMITIVE_TYPE_ERROR_FIELD, LIST_FIELD);
+        }
+
+        GroupType listType = groupType.getType(LIST_FIELD).asGroupType();
+
+        if (!listType.containsField(ELEMENT_FIELD)) {
+            throw new HyracksDataException(ErrorCode.EXTRA_FIELD_IN_RESULT_NOT_FOUND_IN_SCHEMA, ELEMENT_FIELD,
+                    listType.getName());
+        }
+
+        recordConsumer.startGroup();
+
+        if (pointable.getNumberOfChildren() > 0) {
+            recordConsumer.startField(LIST_FIELD, groupType.getFieldIndex(LIST_FIELD));
+
+            for (int i = 0; i < pointable.getNumberOfChildren(); i++) {
+                pointable.nextChild();
+                AbstractLazyVisitablePointable child = pointable.getChildVisitablePointable();
+
+                recordConsumer.startGroup();
+                recordConsumer.startField(ELEMENT_FIELD, listType.getFieldIndex(ELEMENT_FIELD));
+                child.accept(this, listType.getType(ELEMENT_FIELD));
+                recordConsumer.endField(ELEMENT_FIELD, listType.getFieldIndex(ELEMENT_FIELD));
+                recordConsumer.endGroup();
+
+            }
+
+            recordConsumer.endField(LIST_FIELD, groupType.getFieldIndex(LIST_FIELD));
+        }
+
+        recordConsumer.endGroup();
+        return null;
+    }
+
+    @Override
+    public Void visit(FlatLazyVisitablePointable pointable, Type type) throws HyracksDataException {
+
+        if (!type.isPrimitive()) {
+            throw new HyracksDataException(ErrorCode.RESULT_DOES_NOT_FOLLOW_SCHEMA, PRIMITIVE_TYPE_ERROR_FIELD,
+                    GROUP_TYPE_ERROR_FIELD, type.getName());
+        }
+        parquetValueWriter.addValueToColumn(recordConsumer, pointable, type.asPrimitiveType());
+        return null;
+    }
+
+    public void consumeRecord(IValueReference valueReference, RecordConsumer recordConsumer)
+            throws HyracksDataException {
+        rec.set(valueReference);
+        this.recordConsumer = recordConsumer;
+
+        recordConsumer.startMessage();
+        for (int i = 0; i < rec.getNumberOfChildren(); i++) {
+            rec.nextChild();
+            String columnName = fieldNamesDictionary.getOrCreateFieldNameIndex(rec.getFieldName());
+            AbstractLazyVisitablePointable child = rec.getChildVisitablePointable();
+
+            if (!schema.containsField(columnName)) {
+                throw new HyracksDataException(ErrorCode.EXTRA_FIELD_IN_RESULT_NOT_FOUND_IN_SCHEMA, columnName,
+                        schema.getName());
+            }
+
+            recordConsumer.startField(columnName, schema.getFieldIndex(columnName));
+            child.accept(this, schema.getType(columnName));
+            recordConsumer.endField(columnName, schema.getFieldIndex(columnName));
+        }
+        recordConsumer.endMessage();
+    }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/ParquetSchemaBuilderUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/ParquetSchemaBuilderUtils.java
new file mode 100644
index 0000000..da0cef0
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/ParquetSchemaBuilderUtils.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.writer.printer.parquet;
+
+import org.apache.parquet.schema.LogicalTypeAnnotation;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Types;
+
+public class ParquetSchemaBuilderUtils {
+
+    public static Types.BaseGroupBuilder<?, ?> getGroupChild(Types.Builder parent) {
+        if (parent instanceof Types.BaseGroupBuilder) {
+            return ((Types.BaseGroupBuilder<?, ?>) parent).optionalGroup();
+        } else if (parent instanceof Types.BaseListBuilder) {
+            return ((Types.BaseListBuilder<?, ?>) parent).optionalGroupElement();
+        } else {
+            return null;
+        }
+    }
+
+    public static Types.BaseListBuilder<?, ?> getListChild(Types.Builder parent) {
+        if (parent instanceof Types.BaseGroupBuilder) {
+            return ((Types.BaseGroupBuilder<?, ?>) parent).optionalList();
+        } else if (parent instanceof Types.BaseListBuilder) {
+            return ((Types.BaseListBuilder<?, ?>) parent).optionalListElement();
+        } else {
+            return null;
+        }
+    }
+
+    public static Types.Builder<?, ?> getPrimitiveChild(Types.Builder parent, PrimitiveType.PrimitiveTypeName type,
+            LogicalTypeAnnotation annotation) {
+        if (parent instanceof Types.BaseGroupBuilder) {
+            return ((Types.BaseGroupBuilder<?, ?>) parent).optional(type).as(annotation);
+        } else if (parent instanceof Types.BaseListBuilder) {
+            return ((Types.BaseListBuilder<?, ?>) parent).optionalElement(type).as(annotation);
+        } else {
+            return null;
+        }
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/ParquetSchemaLazyVisitor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/ParquetSchemaLazyVisitor.java
new file mode 100644
index 0000000..055c635
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/ParquetSchemaLazyVisitor.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.writer.printer.parquet;
+
+import static org.apache.asterix.external.writer.printer.parquet.ParquetSchemaTree.buildParquetSchema;
+
+import java.util.Map;
+
+import org.apache.asterix.om.lazy.AbstractLazyVisitablePointable;
+import org.apache.asterix.om.lazy.AbstractListLazyVisitablePointable;
+import org.apache.asterix.om.lazy.FlatLazyVisitablePointable;
+import org.apache.asterix.om.lazy.ILazyVisitablePointableVisitor;
+import org.apache.asterix.om.lazy.RecordLazyVisitablePointable;
+import org.apache.asterix.om.lazy.TypedRecordLazyVisitablePointable;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.Types;
+
+public class ParquetSchemaLazyVisitor implements ILazyVisitablePointableVisitor<Void, ParquetSchemaTree.SchemaNode> {
+    private final RecordLazyVisitablePointable rec;
+    private final FieldNamesDictionary fieldNamesDictionary;
+    private final static String SCHEMA_NAME = "asterix_schema";
+
+    public ParquetSchemaLazyVisitor(IAType typeInfo) {
+        this.fieldNamesDictionary = new FieldNamesDictionary();
+        if (typeInfo.getTypeTag() == ATypeTag.OBJECT) {
+            this.rec = new TypedRecordLazyVisitablePointable((ARecordType) typeInfo);
+        } else if (typeInfo.getTypeTag() == ATypeTag.ANY) {
+            this.rec = new RecordLazyVisitablePointable(true);
+        } else {
+            throw new RuntimeException("Type Unsupported for parquet printing");
+        }
+    }
+
+    @Override
+    public Void visit(RecordLazyVisitablePointable pointable, ParquetSchemaTree.SchemaNode schemaNode)
+            throws HyracksDataException {
+        if (schemaNode.getType() == null) {
+            schemaNode.setType(new ParquetSchemaTree.RecordType());
+        }
+        if (!(schemaNode.getType() instanceof ParquetSchemaTree.RecordType)) {
+            throw new HyracksDataException(ErrorCode.RESULT_DOES_NOT_FOLLOW_SCHEMA);
+        }
+        ParquetSchemaTree.RecordType recordType = (ParquetSchemaTree.RecordType) schemaNode.getType();
+        for (int i = 0; i < pointable.getNumberOfChildren(); i++) {
+            pointable.nextChild();
+            AbstractLazyVisitablePointable child = pointable.getChildVisitablePointable();
+            String childColumnName = fieldNamesDictionary.getOrCreateFieldNameIndex(pointable.getFieldName());
+            ParquetSchemaTree.SchemaNode childType;
+            if (recordType.getChildren().containsKey(childColumnName)) {
+                childType = recordType.getChildren().get(childColumnName);
+            } else {
+                childType = new ParquetSchemaTree.SchemaNode();
+                recordType.add(childColumnName, childType);
+            }
+            child.accept(this, childType);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visit(AbstractListLazyVisitablePointable pointable, ParquetSchemaTree.SchemaNode schemaNode)
+            throws HyracksDataException {
+        if (schemaNode.getType() == null) {
+            schemaNode.setType(new ParquetSchemaTree.ListType());
+        }
+        if (!(schemaNode.getType() instanceof ParquetSchemaTree.ListType)) {
+            throw new HyracksDataException(ErrorCode.RESULT_DOES_NOT_FOLLOW_SCHEMA);
+        }
+        ParquetSchemaTree.ListType listType = (ParquetSchemaTree.ListType) schemaNode.getType();
+        for (int i = 0; i < pointable.getNumberOfChildren(); i++) {
+            pointable.nextChild();
+            AbstractLazyVisitablePointable child = pointable.getChildVisitablePointable();
+            if (listType.isEmpty()) {
+                listType.setChild(new ParquetSchemaTree.SchemaNode());
+            }
+            child.accept(this, listType.getChild());
+        }
+        return null;
+    }
+
+    @Override
+    public Void visit(FlatLazyVisitablePointable pointable, ParquetSchemaTree.SchemaNode schemaNode)
+            throws HyracksDataException {
+        if (schemaNode.getType() == null) {
+            schemaNode.setType(new ParquetSchemaTree.FlatType(
+                    AsterixParquetTypeMap.PRIMITIVE_TYPE_NAME_MAP.get(pointable.getTypeTag()),
+                    AsterixParquetTypeMap.LOGICAL_TYPE_ANNOTATION_MAP.get(pointable.getTypeTag())));
+            return null;
+        }
+        if (!(schemaNode.getType() instanceof ParquetSchemaTree.FlatType)) {
+            throw new HyracksDataException(ErrorCode.RESULT_DOES_NOT_FOLLOW_SCHEMA);
+        }
+        ParquetSchemaTree.FlatType flatType = (ParquetSchemaTree.FlatType) schemaNode.getType();
+        if (!(flatType.getPrimitiveTypeName() == AsterixParquetTypeMap.PRIMITIVE_TYPE_NAME_MAP
+                .get(pointable.getTypeTag()))
+                || !(flatType.getLogicalTypeAnnotation() == AsterixParquetTypeMap.LOGICAL_TYPE_ANNOTATION_MAP
+                        .get(pointable.getTypeTag()))) {
+            throw new HyracksDataException(ErrorCode.RESULT_DOES_NOT_FOLLOW_SCHEMA);
+        }
+        return null;
+    }
+
+    public ParquetSchemaTree.SchemaNode inferSchema(IValueReference valueReference) throws HyracksDataException {
+        ParquetSchemaTree.SchemaNode schemaNode = new ParquetSchemaTree.SchemaNode();
+        rec.set(valueReference);
+        rec.accept(this, schemaNode);
+        return schemaNode;
+    }
+
+    public static MessageType generateSchema(ParquetSchemaTree.SchemaNode schemaRoot) throws HyracksDataException {
+        Types.MessageTypeBuilder builder = Types.buildMessage();
+        if (schemaRoot.getType() == null)
+            return builder.named(SCHEMA_NAME);
+        for (Map.Entry<String, ParquetSchemaTree.SchemaNode> entry : ((ParquetSchemaTree.RecordType) schemaRoot
+                .getType()).getChildren().entrySet()) {
+            buildParquetSchema(builder, entry.getValue(), entry.getKey());
+        }
+        return builder.named(SCHEMA_NAME);
+    }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/ParquetSchemaTree.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/ParquetSchemaTree.java
new file mode 100644
index 0000000..ff512df
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/ParquetSchemaTree.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.writer.printer.parquet;
+
+import static org.apache.asterix.external.writer.printer.parquet.ParquetSchemaBuilderUtils.getGroupChild;
+import static org.apache.asterix.external.writer.printer.parquet.ParquetSchemaBuilderUtils.getListChild;
+import static org.apache.asterix.external.writer.printer.parquet.ParquetSchemaBuilderUtils.getPrimitiveChild;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.parquet.schema.LogicalTypeAnnotation;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Types;
+
+public class ParquetSchemaTree {
+
+    public static class SchemaNode {
+        private AbstractType type;
+
+        public SchemaNode() {
+            type = null;
+        }
+
+        public void setType(AbstractType type) {
+            this.type = type;
+        }
+
+        public AbstractType getType() {
+            return type;
+        }
+    }
+
+    static class RecordType extends AbstractType {
+        private final Map<String, SchemaNode> children;
+
+        public RecordType() {
+            children = new HashMap<>();
+        }
+
+        void add(String fieldName, SchemaNode childType) {
+            children.put(fieldName, childType);
+        }
+
+        SchemaNode get(String fieldName) {
+            return children.get(fieldName);
+        }
+
+        Map<String, SchemaNode> getChildren() {
+            return children;
+        }
+    }
+
+    abstract static class AbstractType {
+    }
+
+    static class FlatType extends AbstractType {
+        private final PrimitiveType.PrimitiveTypeName primitiveTypeName;
+        private final LogicalTypeAnnotation logicalTypeAnnotation;
+
+        public FlatType(PrimitiveType.PrimitiveTypeName primitiveTypeName,
+                LogicalTypeAnnotation logicalTypeAnnotation) {
+            this.primitiveTypeName = primitiveTypeName;
+            this.logicalTypeAnnotation = logicalTypeAnnotation;
+        }
+
+        public LogicalTypeAnnotation getLogicalTypeAnnotation() {
+            return logicalTypeAnnotation;
+        }
+
+        public PrimitiveType.PrimitiveTypeName getPrimitiveTypeName() {
+            return primitiveTypeName;
+        }
+    }
+
+    static class ListType extends AbstractType {
+        private SchemaNode child;
+
+        public ListType() {
+            child = null;
+        }
+
+        void setChild(SchemaNode child) {
+            this.child = child;
+        }
+
+        boolean isEmpty() {
+            return child == null;
+        }
+
+        public SchemaNode getChild() {
+            return child;
+        }
+    }
+
+    public static void buildParquetSchema(Types.Builder builder, SchemaNode schemaNode, String columnName)
+            throws HyracksDataException {
+        if (schemaNode.getType() == null) {
+            throw new HyracksDataException(ErrorCode.EMPTY_TYPE_INFERRED);
+        }
+        AbstractType typeClass = schemaNode.getType();
+        if (typeClass instanceof RecordType) {
+            buildRecord(builder, (RecordType) schemaNode.getType(), columnName);
+        } else if (typeClass instanceof ListType) {
+            buildList(builder, (ListType) schemaNode.getType(), columnName);
+        } else if (typeClass instanceof FlatType) {
+            buildFlat(builder, (FlatType) schemaNode.getType(), columnName);
+        }
+    }
+
+    private static void buildRecord(Types.Builder builder, RecordType type, String columnName)
+            throws HyracksDataException {
+        Types.BaseGroupBuilder<?, ?> childBuilder = getGroupChild(builder);
+        for (Map.Entry<String, SchemaNode> entry : type.getChildren().entrySet()) {
+            buildParquetSchema(childBuilder, entry.getValue(), entry.getKey());
+        }
+        childBuilder.named(columnName);
+    }
+
+    private static void buildList(Types.Builder builder, ListType type, String columnName) throws HyracksDataException {
+        Types.BaseListBuilder<?, ?> childBuilder = getListChild(builder);
+        SchemaNode child = type.child;
+        if (child == null) {
+            throw new HyracksDataException(ErrorCode.EMPTY_TYPE_INFERRED);
+        }
+        buildParquetSchema(childBuilder, child, columnName);
+    }
+
+    private static void buildFlat(Types.Builder builder, FlatType type, String columnName) throws HyracksDataException {
+        if (type.getPrimitiveTypeName() == null) {
+            // Not sure if this is the right thing to do here
+            throw new HyracksDataException(ErrorCode.EMPTY_TYPE_INFERRED);
+        }
+        getPrimitiveChild(builder, type.getPrimitiveTypeName(), type.getLogicalTypeAnnotation()).named(columnName);
+    }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/ParquetValueWriter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/ParquetValueWriter.java
new file mode 100644
index 0000000..0390315
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/ParquetValueWriter.java
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.writer.printer.parquet;
+
+import java.io.IOException;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.dataflow.data.nontagged.printers.PrintTools;
+import org.apache.asterix.dataflow.data.nontagged.serde.ABooleanSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ATimeSerializerDeserializer;
+import org.apache.asterix.om.lazy.FlatLazyVisitablePointable;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.utils.ResettableByteArrayOutputStream;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.util.string.UTF8StringUtil;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.io.api.RecordConsumer;
+import org.apache.parquet.schema.PrimitiveType;
+
+public class ParquetValueWriter {
+    public static final String LIST_FIELD = "list";
+    public static final String ELEMENT_FIELD = "element";
+
+    public static final String GROUP_TYPE_ERROR_FIELD = "group";
+    public static final String PRIMITIVE_TYPE_ERROR_FIELD = "primitive";
+
+    private final VoidPointable voidPointable;
+    private final ResettableByteArrayOutputStream byteArrayOutputStream;
+
+    ParquetValueWriter() {
+        this.voidPointable = VoidPointable.FACTORY.createPointable();
+        this.byteArrayOutputStream = new ResettableByteArrayOutputStream();
+    }
+
+    private void addIntegerType(long value, PrimitiveType.PrimitiveTypeName primitiveTypeName, ATypeTag typeTag,
+            RecordConsumer recordConsumer) throws HyracksDataException {
+        switch (primitiveTypeName) {
+            case INT32:
+                recordConsumer.addInteger((int) value);
+                break;
+            case INT64:
+                recordConsumer.addLong(value);
+                break;
+            case FLOAT:
+                recordConsumer.addFloat(value);
+                break;
+            case DOUBLE:
+                recordConsumer.addDouble(value);
+                break;
+            default:
+                throw RuntimeDataException.create(ErrorCode.TYPE_MISMATCH_GENERIC, primitiveTypeName, typeTag);
+        }
+    }
+
+    public void addValueToColumn(RecordConsumer recordConsumer, FlatLazyVisitablePointable pointable,
+            PrimitiveType type) throws HyracksDataException {
+
+        ATypeTag typeTag = pointable.getTypeTag();
+        byte[] b = pointable.getByteArray();
+        int s, l;
+
+        if (pointable.isTagged()) {
+            s = pointable.getStartOffset() + 1;
+            l = pointable.getLength() - 1;
+        } else {
+            s = pointable.getStartOffset();
+            l = pointable.getLength();
+        }
+        voidPointable.set(b, s, l);
+
+        PrimitiveType.PrimitiveTypeName primitiveTypeName = type.getPrimitiveTypeName();
+
+        switch (typeTag) {
+            case TINYINT:
+                byte tinyIntValue = AInt8SerializerDeserializer.getByte(b, s);
+                addIntegerType(tinyIntValue, primitiveTypeName, typeTag, recordConsumer);
+                break;
+            case SMALLINT:
+                short smallIntValue = AInt16SerializerDeserializer.getShort(b, s);
+                addIntegerType(smallIntValue, primitiveTypeName, typeTag, recordConsumer);
+                break;
+            case INTEGER:
+                int intValue = AInt32SerializerDeserializer.getInt(b, s);
+                addIntegerType(intValue, primitiveTypeName, typeTag, recordConsumer);
+                break;
+            case BIGINT:
+                long bigIntValue = AInt64SerializerDeserializer.getLong(b, s);
+                addIntegerType(bigIntValue, primitiveTypeName, typeTag, recordConsumer);
+                break;
+            case FLOAT:
+                float floatValue = AFloatSerializerDeserializer.getFloat(b, s);
+                switch (primitiveTypeName) {
+                    case INT32:
+                        recordConsumer.addInteger((int) floatValue);
+                        break;
+                    case INT64:
+                        recordConsumer.addLong((long) floatValue);
+                        break;
+                    case FLOAT:
+                        recordConsumer.addFloat(floatValue);
+                        break;
+                    case DOUBLE:
+                        recordConsumer.addDouble(floatValue);
+                        break;
+                    default:
+                        throw RuntimeDataException.create(ErrorCode.TYPE_MISMATCH_GENERIC, primitiveTypeName, typeTag);
+                }
+                break;
+            case DOUBLE:
+                double doubleValue = ADoubleSerializerDeserializer.getDouble(b, s);
+                switch (primitiveTypeName) {
+                    case INT32:
+                        recordConsumer.addInteger((int) doubleValue);
+                        break;
+                    case INT64:
+                        recordConsumer.addLong((long) doubleValue);
+                        break;
+                    case FLOAT:
+                        recordConsumer.addFloat((float) doubleValue);
+                        break;
+                    case DOUBLE:
+                        recordConsumer.addDouble(doubleValue);
+                        break;
+                    default:
+                        throw RuntimeDataException.create(ErrorCode.TYPE_MISMATCH_GENERIC, primitiveTypeName, typeTag);
+                }
+                break;
+            case STRING:
+                int utfLength = UTF8StringUtil.getUTFLength(b, s);
+                if (primitiveTypeName == PrimitiveType.PrimitiveTypeName.BINARY) {
+                    byteArrayOutputStream.reset();
+                    try {
+                        PrintTools.writeUTF8StringAsJSONUnquoted(b, s, l, utfLength, byteArrayOutputStream);
+                    } catch (IOException e) {
+                        throw HyracksDataException.create(e);
+                    }
+                    recordConsumer.addBinary(Binary.fromReusedByteArray(byteArrayOutputStream.getByteArray(), 0,
+                            byteArrayOutputStream.getLength()));
+
+                } else {
+                    throw RuntimeDataException.create(ErrorCode.TYPE_MISMATCH_GENERIC, primitiveTypeName, typeTag);
+                }
+                break;
+            case BOOLEAN:
+                boolean booleanValue = ABooleanSerializerDeserializer.getBoolean(b, s);
+                if (primitiveTypeName == PrimitiveType.PrimitiveTypeName.BOOLEAN) {
+                    recordConsumer.addBoolean(booleanValue);
+                } else {
+                    throw RuntimeDataException.create(ErrorCode.TYPE_MISMATCH_GENERIC, primitiveTypeName, typeTag);
+                }
+                break;
+            case DATE:
+                int dateValue = ADateSerializerDeserializer.getChronon(b, s);
+                addIntegerType(dateValue, primitiveTypeName, typeTag, recordConsumer);
+                break;
+            case TIME:
+                int timeValue = ATimeSerializerDeserializer.getChronon(b, s);
+                addIntegerType(timeValue, primitiveTypeName, typeTag, recordConsumer);
+                break;
+            case DATETIME:
+                long dateTimeValue = ADateTimeSerializerDeserializer.getChronon(b, s);
+                addIntegerType(dateTimeValue, primitiveTypeName, typeTag, recordConsumer);
+            case NULL:
+            case MISSING:
+                break;
+            default:
+                throw RuntimeDataException.create(ErrorCode.TYPE_MISMATCH_GENERIC, primitiveTypeName, typeTag);
+        }
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/SchemaCheckerLazyVisitor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/SchemaCheckerLazyVisitor.java
new file mode 100644
index 0000000..44cd5b2
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/SchemaCheckerLazyVisitor.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.writer.printer.parquet;
+
+import org.apache.asterix.om.lazy.AbstractLazyVisitablePointable;
+import org.apache.asterix.om.lazy.AbstractListLazyVisitablePointable;
+import org.apache.asterix.om.lazy.FlatLazyVisitablePointable;
+import org.apache.asterix.om.lazy.ILazyVisitablePointableVisitor;
+import org.apache.asterix.om.lazy.RecordLazyVisitablePointable;
+import org.apache.asterix.om.lazy.TypedRecordLazyVisitablePointable;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+public class SchemaCheckerLazyVisitor implements ISchemaChecker,
+        ILazyVisitablePointableVisitor<ISchemaChecker.SchemaComparisonType, ParquetSchemaTree.SchemaNode> {
+    private final FieldNamesDictionary fieldNamesDictionary;
+    private final RecordLazyVisitablePointable record;
+
+    public SchemaCheckerLazyVisitor(IAType typeInfo) {
+        this.fieldNamesDictionary = new FieldNamesDictionary();
+        if (typeInfo.getTypeTag() == ATypeTag.OBJECT) {
+            this.record = new TypedRecordLazyVisitablePointable((ARecordType) typeInfo);
+        } else if (typeInfo.getTypeTag() == ATypeTag.ANY) {
+            this.record = new RecordLazyVisitablePointable(true);
+        } else {
+            throw new RuntimeException("Type Unsupported for parquet printing");
+        }
+    }
+
+    @Override
+    public ISchemaChecker.SchemaComparisonType visit(RecordLazyVisitablePointable pointable,
+            ParquetSchemaTree.SchemaNode schemaNode) throws HyracksDataException {
+        if (schemaNode.getType() == null) {
+            return ISchemaChecker.SchemaComparisonType.GROWING;
+        }
+
+        if (!(schemaNode.getType() instanceof ParquetSchemaTree.RecordType)) {
+            return ISchemaChecker.SchemaComparisonType.CONFLICTING;
+        }
+
+        ParquetSchemaTree.RecordType recordType = (ParquetSchemaTree.RecordType) schemaNode.getType();
+        ISchemaChecker.SchemaComparisonType schemaComparisonType = ISchemaChecker.SchemaComparisonType.EQUIVALENT;
+
+        for (int i = 0; i < pointable.getNumberOfChildren(); i++) {
+            pointable.nextChild();
+            AbstractLazyVisitablePointable child = pointable.getChildVisitablePointable();
+            String childColumnName = fieldNamesDictionary.getOrCreateFieldNameIndex(pointable.getFieldName());
+            ParquetSchemaTree.SchemaNode childType = recordType.getChildren().get(childColumnName);
+            if (childType == null) {
+                schemaComparisonType =
+                        ISchemaChecker.max(schemaComparisonType, ISchemaChecker.SchemaComparisonType.GROWING);
+                continue;
+            }
+            schemaComparisonType = ISchemaChecker.max(schemaComparisonType, child.accept(this, childType));
+        }
+        return schemaComparisonType;
+    }
+
+    @Override
+    public ISchemaChecker.SchemaComparisonType visit(AbstractListLazyVisitablePointable pointable,
+            ParquetSchemaTree.SchemaNode schemaNode) throws HyracksDataException {
+        if (schemaNode.getType() == null) {
+            return ISchemaChecker.SchemaComparisonType.GROWING;
+        }
+        if (!(schemaNode.getType() instanceof ParquetSchemaTree.ListType)) {
+            return ISchemaChecker.SchemaComparisonType.CONFLICTING;
+        }
+
+        ParquetSchemaTree.ListType listType = (ParquetSchemaTree.ListType) schemaNode.getType();
+        ISchemaChecker.SchemaComparisonType schemaComparisonType = ISchemaChecker.SchemaComparisonType.EQUIVALENT;
+
+        for (int i = 0; i < pointable.getNumberOfChildren(); i++) {
+            pointable.nextChild();
+            AbstractLazyVisitablePointable child = pointable.getChildVisitablePointable();
+            if (listType.isEmpty()) {
+                schemaComparisonType =
+                        ISchemaChecker.max(schemaComparisonType, ISchemaChecker.SchemaComparisonType.GROWING);
+                continue;
+            }
+            schemaComparisonType = ISchemaChecker.max(schemaComparisonType, child.accept(this, listType.getChild()));
+        }
+        return schemaComparisonType;
+    }
+
+    @Override
+    public ISchemaChecker.SchemaComparisonType visit(FlatLazyVisitablePointable pointable,
+            ParquetSchemaTree.SchemaNode schemaNode) throws HyracksDataException {
+        if (schemaNode.getType() == null) {
+            return ISchemaChecker.SchemaComparisonType.GROWING;
+        }
+        if (!(schemaNode.getType() instanceof ParquetSchemaTree.FlatType)) {
+            return ISchemaChecker.SchemaComparisonType.CONFLICTING;
+        }
+
+        ParquetSchemaTree.FlatType flatType = (ParquetSchemaTree.FlatType) schemaNode.getType();
+
+        if (flatType.getPrimitiveTypeName() != AsterixParquetTypeMap.PRIMITIVE_TYPE_NAME_MAP.get(pointable.getTypeTag())
+                || !(flatType.getLogicalTypeAnnotation() == AsterixParquetTypeMap.LOGICAL_TYPE_ANNOTATION_MAP
+                        .get(pointable.getTypeTag()))) {
+            return ISchemaChecker.SchemaComparisonType.CONFLICTING;
+        }
+
+        return ISchemaChecker.SchemaComparisonType.EQUIVALENT;
+    }
+
+    @Override
+    public SchemaComparisonType checkSchema(ParquetSchemaTree.SchemaNode schemaNode, IValueReference valueReference)
+            throws HyracksDataException {
+        record.set(valueReference);
+        return record.accept(this, schemaNode);
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/SchemaConverterVisitor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/SchemaConverterVisitor.java
new file mode 100644
index 0000000..a6ea115
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/parquet/SchemaConverterVisitor.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.writer.printer.parquet;
+
+import static org.apache.asterix.external.writer.printer.parquet.ParquetSchemaBuilderUtils.getGroupChild;
+import static org.apache.asterix.external.writer.printer.parquet.ParquetSchemaBuilderUtils.getListChild;
+import static org.apache.asterix.external.writer.printer.parquet.ParquetSchemaBuilderUtils.getPrimitiveChild;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.IATypeVisitor;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.parquet.schema.LogicalTypeAnnotation;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Types;
+
+public class SchemaConverterVisitor implements IATypeVisitor<Void, Pair<Types.Builder, String>> {
+    public static String MESSAGE_NAME = "asterix_schema";
+    private final ARecordType schemaType;
+    private ATypeTag unsupportedType;
+
+    private SchemaConverterVisitor(ARecordType schemaType) {
+        this.schemaType = schemaType;
+        this.unsupportedType = null;
+    }
+
+    public static String convertToParquetSchemaString(ARecordType schemaType) throws CompilationException {
+        SchemaConverterVisitor schemaConverterVisitor = new SchemaConverterVisitor(schemaType);
+        return schemaConverterVisitor.getParquetSchema().toString();
+    }
+
+    private MessageType getParquetSchema() throws CompilationException {
+        Types.MessageTypeBuilder builder = Types.buildMessage();
+
+        for (int i = 0; i < schemaType.getFieldNames().length; i++) {
+            String fieldName = schemaType.getFieldNames()[i];
+            IAType childType = schemaType.getFieldType(fieldName);
+            childType.accept(this, new Pair<>(builder, fieldName));
+            if (unsupportedType != null) {
+                throw new CompilationException(ErrorCode.TYPE_UNSUPPORTED_PARQUET_WRITE, unsupportedType.toString());
+            }
+        }
+        return builder.named(MESSAGE_NAME);
+    }
+
+    @Override
+    public Void visit(ARecordType recordType, Pair<Types.Builder, String> arg) {
+        Types.Builder builder = arg.first;
+        String fieldName = arg.second;
+
+        Types.BaseGroupBuilder childBuilder = getGroupChild(builder);
+        for (int i = 0; i < recordType.getFieldNames().length; i++) {
+            String childFieldName = recordType.getFieldNames()[i];
+            IAType childType = recordType.getFieldType(childFieldName);
+
+            childType.accept(this, new Pair<>(childBuilder, childFieldName));
+
+        }
+        childBuilder.named(fieldName);
+
+        return null;
+    }
+
+    @Override
+    public Void visit(AbstractCollectionType collectionType, Pair<Types.Builder, String> arg) {
+        Types.Builder builder = arg.first;
+        String fieldName = arg.second;
+
+        Types.BaseListBuilder childBuilder = getListChild(builder);
+        IAType child = collectionType.getItemType();
+        child.accept(this, new Pair<>(childBuilder, fieldName));
+
+        return null;
+    }
+
+    @Override
+    public Void visit(AUnionType unionType, Pair<Types.Builder, String> arg) {
+        // Shouldn't reach here.
+        return null;
+    }
+
+    @Override
+    public Void visitFlat(IAType flatType, Pair<Types.Builder, String> arg) {
+        Types.Builder builder = arg.first;
+        String fieldName = arg.second;
+
+        PrimitiveType.PrimitiveTypeName primitiveTypeName =
+                AsterixParquetTypeMap.PRIMITIVE_TYPE_NAME_MAP.get(flatType.getTypeTag());
+
+        if (primitiveTypeName == null) {
+            unsupportedType = flatType.getTypeTag();
+        }
+
+        LogicalTypeAnnotation logicalTypeAnnotation =
+                AsterixParquetTypeMap.LOGICAL_TYPE_ANNOTATION_MAP.get(flatType.getTypeTag());
+
+        getPrimitiveChild(builder, primitiveTypeName, logicalTypeAnnotation).named(fieldName);
+
+        return null;
+    }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IRecordReaderFactory b/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IRecordReaderFactory
index 2c15b5a..1f25c4b 100644
--- a/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IRecordReaderFactory
+++ b/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IRecordReaderFactory
@@ -28,4 +28,5 @@
 org.apache.asterix.external.input.record.reader.azure.datalake.AzureDataLakeReaderFactory
 org.apache.asterix.external.input.record.reader.azure.parquet.AzureBlobParquetReaderFactory
 org.apache.asterix.external.input.record.reader.azure.parquet.AzureDataLakeParquetReaderFactory
-org.apache.asterix.external.input.record.reader.gcs.parquet.GCSParquetReaderFactory
\ No newline at end of file
+org.apache.asterix.external.input.record.reader.gcs.parquet.GCSParquetReaderFactory
+org.apache.asterix.external.input.record.reader.gcs.delta.GCSDeltaReaderFactory
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/awss3/DeltaTopicPartitionDistributionTest.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/awss3/DeltaTopicPartitionDistributionTest.java
new file mode 100644
index 0000000..1bc8eb8
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/awss3/DeltaTopicPartitionDistributionTest.java
@@ -0,0 +1,342 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader.awss3;
+
+import static io.delta.kernel.internal.InternalScanFileUtils.ADD_FILE_ORDINAL;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.external.input.record.reader.aws.delta.DeltaReaderFactory;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.junit.Assert;
+import org.junit.Test;
+
+import io.delta.kernel.data.ArrayValue;
+import io.delta.kernel.data.MapValue;
+import io.delta.kernel.data.Row;
+import io.delta.kernel.types.IntegerType;
+import io.delta.kernel.types.StringType;
+import io.delta.kernel.types.StructField;
+import io.delta.kernel.types.StructType;
+
+public class DeltaTopicPartitionDistributionTest {
+
+    @Test
+    public void distributeFilesMoreFilesThanPartitions() {
+        int rowCount = 25;
+        int numberOfPartition = 13;
+        List<Row> scanFiles = createMockRows(rowCount);
+        DeltaReaderFactory d = new DeltaReaderFactory() {
+            @Override
+            protected void configureJobConf(JobConf conf, Map<String, String> configuration)
+                    throws AlgebricksException {
+
+            }
+
+            @Override
+            protected String getTablePath(Map<String, String> configuration) throws AlgebricksException {
+                return null;
+            }
+
+            @Override
+            public List<String> getRecordReaderNames() {
+                return null;
+            }
+        };
+        d.distributeFiles(scanFiles, numberOfPartition);
+        Assert.assertEquals(numberOfPartition, d.getPartitionWorkLoadsBasedOnSize().size());
+        verifyFileDistribution(scanFiles.size(), d.getPartitionWorkLoadsBasedOnSize());
+    }
+
+    @Test
+    public void distributeFilesLessFilesThanPartitions() {
+        int rowCount = 15;
+        int numberOfPartition = 23;
+        List<Row> scanFiles = createMockRows(rowCount);
+        DeltaReaderFactory d = new DeltaReaderFactory() {
+            @Override
+            protected void configureJobConf(JobConf conf, Map<String, String> configuration)
+                    throws AlgebricksException {
+
+            }
+
+            @Override
+            protected String getTablePath(Map<String, String> configuration) throws AlgebricksException {
+                return null;
+            }
+
+            @Override
+            public List<String> getRecordReaderNames() {
+                return null;
+            }
+        };
+        d.distributeFiles(scanFiles, numberOfPartition);
+        Assert.assertEquals(numberOfPartition, d.getPartitionWorkLoadsBasedOnSize().size());
+        verifyFileDistribution(scanFiles.size(), d.getPartitionWorkLoadsBasedOnSize());
+    }
+
+    @Test
+    public void distributeFilesEqualFilesAndPartitions() {
+        int rowCount = 9;
+        int numberOfPartition = 9;
+        List<Row> scanFiles = createMockRows(rowCount);
+        DeltaReaderFactory d = new DeltaReaderFactory() {
+            @Override
+            protected void configureJobConf(JobConf conf, Map<String, String> configuration)
+                    throws AlgebricksException {
+
+            }
+
+            @Override
+            protected String getTablePath(Map<String, String> configuration) throws AlgebricksException {
+                return null;
+            }
+
+            @Override
+            public List<String> getRecordReaderNames() {
+                return null;
+            }
+        };
+        d.distributeFiles(scanFiles, numberOfPartition);
+        Assert.assertEquals(numberOfPartition, d.getPartitionWorkLoadsBasedOnSize().size());
+        verifyFileDistribution(scanFiles.size(), d.getPartitionWorkLoadsBasedOnSize());
+    }
+
+    private void verifyFileDistribution(int numberOfFiles,
+            List<DeltaReaderFactory.PartitionWorkLoadBasedOnSize> workloads) {
+        int totalDistributedFiles = 0;
+
+        for (DeltaReaderFactory.PartitionWorkLoadBasedOnSize workload : workloads) {
+            totalDistributedFiles += workload.getScanFiles().size();
+            Assert.assertTrue(workload.getTotalSize() >= 0);
+        }
+        Assert.assertEquals(numberOfFiles, totalDistributedFiles);
+    }
+
+    private List<Row> createMockRows(int count) {
+        List<Row> rows = new ArrayList<>();
+        StructType sch = createMockSchema();
+
+        for (int i = 1; i <= count; i++) {
+            int finalI = i;
+            Row row = new Row() {
+
+                @Override
+                public StructType getSchema() {
+                    return sch;
+                }
+
+                @Override
+                public boolean isNullAt(int i) {
+                    return false;
+                }
+
+                @Override
+                public boolean getBoolean(int i) {
+                    return false;
+                }
+
+                @Override
+                public byte getByte(int i) {
+                    return 0;
+                }
+
+                @Override
+                public short getShort(int i) {
+                    return 0;
+                }
+
+                @Override
+                public int getInt(int i) {
+                    if (i == 1) {
+                        return finalI;
+                    } else if (i == 2) {
+                        return finalI * 10;
+                    }
+                    return 0;
+                }
+
+                @Override
+                public long getLong(int i) {
+                    return 0;
+                }
+
+                @Override
+                public float getFloat(int i) {
+                    return 0;
+                }
+
+                @Override
+                public double getDouble(int i) {
+                    return 0;
+                }
+
+                @Override
+                public String getString(int i) {
+                    if (i == 0) {
+                        return "tableRoot_" + finalI;
+                    } else if (i == 1) {
+                        return "addFilePath_" + finalI;
+                    }
+                    return null;
+                }
+
+                @Override
+                public BigDecimal getDecimal(int i) {
+                    return null;
+                }
+
+                @Override
+                public byte[] getBinary(int i) {
+                    return new byte[0];
+                }
+
+                @Override
+                public ArrayValue getArray(int i) {
+                    return null;
+                }
+
+                @Override
+                public MapValue getMap(int i) {
+                    return null;
+                }
+
+                @Override
+                public Row getStruct(int index) {
+                    if (index == ADD_FILE_ORDINAL) {
+                        return createAddFileEntry(finalI);
+                    }
+                    return null;
+                }
+            };
+
+            rows.add(row);
+        }
+
+        return rows;
+    }
+
+    private StructType createMockSchema() {
+        List<StructField> fields = new ArrayList<>();
+
+        fields.add(new StructField("field1", StringType.STRING, true));
+        fields.add(new StructField("field2", IntegerType.INTEGER, true));
+        fields.add(new StructField("field3", IntegerType.INTEGER, true));
+
+        return new StructType(fields);
+    }
+
+    private Row createAddFileEntry(int i) {
+        List<StructField> addFileFields = new ArrayList<>();
+
+        addFileFields.add(new StructField("addFilePath", StringType.STRING, true));
+        addFileFields.add(new StructField("size", IntegerType.INTEGER, true));
+
+        StructType addFileSchema = new StructType(addFileFields);
+
+        Row addFileRow = new Row() {
+            @Override
+            public StructType getSchema() {
+                return addFileSchema;
+            }
+
+            @Override
+            public boolean isNullAt(int index) {
+                return false;
+            }
+
+            @Override
+            public boolean getBoolean(int i) {
+                return false;
+            }
+
+            @Override
+            public byte getByte(int i) {
+                return 0;
+            }
+
+            @Override
+            public short getShort(int i) {
+                return 0;
+            }
+
+            @Override
+            public int getInt(int index) {
+                if (index == 1) {
+                    return i * 100;
+                }
+                return 0;
+            }
+
+            @Override
+            public long getLong(int i) {
+                return 0;
+            }
+
+            @Override
+            public float getFloat(int i) {
+                return 0;
+            }
+
+            @Override
+            public double getDouble(int i) {
+                return 0;
+            }
+
+            @Override
+            public BigDecimal getDecimal(int i) {
+                return null;
+            }
+
+            @Override
+            public byte[] getBinary(int i) {
+                return new byte[0];
+            }
+
+            @Override
+            public Row getStruct(int index) {
+                return null;
+            }
+
+            @Override
+            public ArrayValue getArray(int index) {
+                return null;
+            }
+
+            @Override
+            public MapValue getMap(int index) {
+                return null;
+            }
+
+            @Override
+            public String getString(int index) {
+                if (index == 0) {
+                    return "addFilePath_" + i;
+                }
+                return null;
+            }
+        };
+
+        return addFileRow;
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/ADMDataParserTest.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/ADMDataParserTest.java
index 189c842..f6da684 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/ADMDataParserTest.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/ADMDataParserTest.java
@@ -45,8 +45,7 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.junit.Assert;
 import org.junit.Test;
-
-import com.esri.core.geometry.ogc.OGCPoint;
+import org.locationtech.jts.geom.Point;
 
 public class ADMDataParserTest {
 
@@ -192,10 +191,10 @@
                     SerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(BuiltinType.AGEOMETRY);
             Object point = geomDeser.deserialize(dataIn);
             Assert.assertTrue(point instanceof AGeometry);
-            Assert.assertTrue(((AGeometry) point).getGeometry() instanceof OGCPoint);
-            OGCPoint p = (OGCPoint) ((AGeometry) point).getGeometry();
-            Assert.assertEquals(3.0, p.X(), 1E-5);
-            Assert.assertEquals(4.0, p.Y(), 1E-5);
+            Assert.assertTrue(((AGeometry) point).getGeometry() instanceof Point);
+            Point p = (Point) ((AGeometry) point).getGeometry();
+            Assert.assertEquals(3.0, p.getX(), 1E-5);
+            Assert.assertEquals(4.0, p.getY(), 1E-5);
         } catch (IOException e) {
             e.printStackTrace();
             Assert.fail("Error in parsing");
diff --git a/asterixdb/asterix-geo/pom.xml b/asterixdb/asterix-geo/pom.xml
index e4cd8ff..7568d5d 100644
--- a/asterixdb/asterix-geo/pom.xml
+++ b/asterixdb/asterix-geo/pom.xml
@@ -121,8 +121,8 @@
       <version>${project.version}</version>
     </dependency>
     <dependency>
-      <groupId>com.esri.geometry</groupId>
-      <artifactId>esri-geometry-api</artifactId>
+      <groupId>org.locationtech.jts</groupId>
+      <artifactId>jts-core</artifactId>
     </dependency>
   </dependencies>
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateFunction.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateFunction.java
index 4676851..5d64e9d 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateFunction.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateFunction.java
@@ -42,11 +42,9 @@
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-
-import com.esri.core.geometry.Point;
-import com.esri.core.geometry.SpatialReference;
-import com.esri.core.geometry.ogc.OGCGeometry;
-import com.esri.core.geometry.ogc.OGCPoint;
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
 
 /**
  * STUnion aggregates a set of objects into one object. If the input is a set of overlapping polygons, their union is
@@ -55,14 +53,13 @@
  * output is a GeometryCollection.
  */
 public class STUnionAggregateFunction extends AbstractAggregateFunction {
-    /**Use WGS 84 (EPSG:4326) as the default coordinate reference system*/
-    public static final SpatialReference DEFAULT_CRS = SpatialReference.create(4326);
     @SuppressWarnings("unchecked")
     private ISerializerDeserializer<AGeometry> geometrySerde =
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AGEOMETRY);
     private IPointable inputVal = new VoidPointable();
     private IScalarEvaluator eval;
-    protected OGCGeometry geometry;
+    protected Geometry geometry;
+    protected GeometryFactory geometryFactory;
 
     private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
 
@@ -75,7 +72,8 @@
     @Override
     public void init() throws HyracksDataException {
         // Initialize the resulting geometry with an empty point.
-        geometry = new OGCPoint(new Point(), DEFAULT_CRS);
+        geometryFactory = new GeometryFactory();
+        geometry = geometryFactory.createPoint((Coordinate) null);
     }
 
     @Override
@@ -91,7 +89,7 @@
             processNull();
         } else if (typeTag == ATypeTag.GEOMETRY) {
             DataInput dataIn = new DataInputStream(new ByteArrayInputStream(data, offset + 1, len - 1));
-            OGCGeometry geometry1 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn).getGeometry();
+            Geometry geometry1 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn).getGeometry();
             geometry = geometry.union(geometry1);
         }
     }
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionRegistrant.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionRegistrant.java
index 0e4d748..75927cc 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionRegistrant.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionRegistrant.java
@@ -87,6 +87,9 @@
 import org.apache.asterix.om.functions.IFunctionRegistrant;
 
 public class GeoFunctionRegistrant implements IFunctionRegistrant {
+
+    private static final long serialVersionUID = 5815308952562578644L;
+
     @Override
     public void register(IFunctionCollection fc) {
         //Geo functions
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractGetValDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractGetValDescriptor.java
index 65992f7..dd0e9f4 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractGetValDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractGetValDescriptor.java
@@ -25,6 +25,8 @@
 import org.apache.asterix.runtime.exceptions.TypeMismatchException;
 
 public abstract class AbstractGetValDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 5267623277804669828L;
+
     public double getVal(byte[] bytes, int offset) throws TypeMismatchException {
         if (bytes[offset] == ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
             return ADoubleSerializerDeserializer.getDouble(bytes, offset + 1);
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTDoubleGeometryDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTDoubleGeometryDescriptor.java
index 97c30a1..69f841c 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTDoubleGeometryDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTDoubleGeometryDescriptor.java
@@ -26,7 +26,6 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.om.base.ABoolean;
-import org.apache.asterix.om.base.AGeometry;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.EnumDeserializer;
@@ -40,13 +39,12 @@
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public abstract class AbstractSTDoubleGeometryDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = -4181892424621765426L;
 
-    abstract protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1)
-            throws HyracksDataException;
+    abstract protected Object evaluateOGCGeometry(Geometry geometry0, Geometry geometry1) throws HyracksDataException;
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -106,13 +104,13 @@
                 }
 
                 DataInputStream dataIn0 = new DataInputStream(new ByteArrayInputStream(bytes0, offset0 + 1, len0 - 1));
-                OGCGeometry geometry0 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn0).getGeometry();
+                Geometry geometry0 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn0).getGeometry();
                 DataInputStream dataIn1 = new DataInputStream(new ByteArrayInputStream(bytes1, offset1 + 1, len1 - 1));
-                OGCGeometry geometry1 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn1).getGeometry();
+                Geometry geometry1 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn1).getGeometry();
                 Object finalResult = evaluateOGCGeometry(geometry0, geometry1);
-                if (finalResult instanceof OGCGeometry) {
+                if (finalResult instanceof Geometry) {
                     out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
-                    AGeometrySerializerDeserializer.INSTANCE.serialize(new AGeometry((OGCGeometry) finalResult), out);
+                    AGeometrySerializerDeserializer.INSTANCE.serialize((Geometry) finalResult, out);
                 } else if (finalResult instanceof Boolean) {
                     SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN)
                             .serialize((boolean) finalResult ? ABoolean.TRUE : ABoolean.FALSE, out);
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTGeometryNDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTGeometryNDescriptor.java
index 1420214..5d281ec 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTGeometryNDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTGeometryNDescriptor.java
@@ -25,7 +25,6 @@
 
 import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
-import org.apache.asterix.om.base.AGeometry;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
@@ -38,14 +37,13 @@
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public abstract class AbstractSTGeometryNDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
 
-    abstract protected OGCGeometry evaluateOGCGeometry(OGCGeometry geometry, int n) throws HyracksDataException;
+    abstract protected Geometry evaluateOGCGeometry(Geometry geometry, int n) throws HyracksDataException;
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -101,13 +99,13 @@
 
             ByteArrayInputStream inStream = new ByteArrayInputStream(data, offset + 1, len - 1);
             DataInputStream dataIn = new DataInputStream(inStream);
-            OGCGeometry geometry = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn).getGeometry();
+            Geometry geometry = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn).getGeometry();
             int n = (int) AInt64SerializerDeserializer.getLong(data0, offset0 + 1);
 
-            OGCGeometry geometryN = evaluateOGCGeometry(geometry, n);
+            Geometry geometryN = evaluateOGCGeometry(geometry, n);
             try {
                 out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
-                AGeometrySerializerDeserializer.INSTANCE.serialize(new AGeometry(geometryN), out);
+                AGeometrySerializerDeserializer.INSTANCE.serialize(geometryN, out);
                 result.set(resultStorage);
             } catch (IOException e) {
                 throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTSingleGeometryDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTSingleGeometryDescriptor.java
index e41aebb..a1467d7 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTSingleGeometryDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTSingleGeometryDescriptor.java
@@ -27,7 +27,6 @@
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.om.base.ABinary;
 import org.apache.asterix.om.base.ABoolean;
-import org.apache.asterix.om.base.AGeometry;
 import org.apache.asterix.om.base.AMutableInt32;
 import org.apache.asterix.om.base.ARectangle;
 import org.apache.asterix.om.types.ATypeTag;
@@ -44,12 +43,12 @@
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 import org.apache.hyracks.util.string.UTF8StringUtil;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public abstract class AbstractSTSingleGeometryDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = -6601497277041936500L;
 
-    abstract protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException;
+    abstract protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException;
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -99,7 +98,7 @@
                 }
 
                 DataInputStream dataIn0 = new DataInputStream(new ByteArrayInputStream(bytes0, offset0 + 1, len0 - 1));
-                OGCGeometry geometry0 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn0).getGeometry();
+                Geometry geometry0 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn0).getGeometry();
 
                 Object finalResult = evaluateOGCGeometry(geometry0);
                 if (finalResult == null) {
@@ -120,9 +119,9 @@
                 } else if (finalResult instanceof byte[]) {
                     SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABINARY)
                             .serialize(new ABinary((byte[]) finalResult), out);
-                } else if (finalResult instanceof OGCGeometry) {
+                } else if (finalResult instanceof Geometry) {
                     out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
-                    AGeometrySerializerDeserializer.INSTANCE.serialize(new AGeometry((OGCGeometry) finalResult), out);
+                    AGeometrySerializerDeserializer.INSTANCE.serialize((Geometry) finalResult, out);
                 } else if (finalResult instanceof ARectangle) {
                     SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ARECTANGLE)
                             .serialize(finalResult, out);
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/ParseGeoJSONDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/ParseGeoJSONDescriptor.java
index 425a198..8df57e4 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/ParseGeoJSONDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/ParseGeoJSONDescriptor.java
@@ -23,9 +23,10 @@
 import java.io.DataInputStream;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.nio.ByteBuffer;
 
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.JtsModule;
 import org.apache.asterix.geo.evaluators.GeoFunctionTypeInferers;
 import org.apache.asterix.om.base.AOrderedList;
 import org.apache.asterix.om.base.ARecord;
@@ -48,10 +49,9 @@
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import org.locationtech.jts.geom.Geometry;
 
-import com.esri.core.geometry.MapOGCStructure;
-import com.esri.core.geometry.OperatorImportFromGeoJson;
-import com.esri.core.geometry.ogc.OGCGeometry;
+import com.fasterxml.jackson.databind.ObjectMapper;
 
 public class ParseGeoJSONDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
@@ -98,7 +98,7 @@
         private DataOutput out;
         private IPointable inputArg;
         private IScalarEvaluator eval;
-        private OperatorImportFromGeoJson geoJsonImporter;
+        private ObjectMapper objectMapper;
 
         public ParseGeoJSONEvaluator(IScalarEvaluatorFactory factory, IEvaluatorContext ctx)
                 throws HyracksDataException {
@@ -106,7 +106,8 @@
             out = resultStorage.getDataOutput();
             inputArg = new VoidPointable();
             eval = factory.createScalarEvaluator(ctx);
-            geoJsonImporter = OperatorImportFromGeoJson.local();
+            objectMapper = new ObjectMapper();
+            objectMapper.registerModule(new JtsModule());
         }
 
         @Override
@@ -123,15 +124,10 @@
             ByteArrayInputStream inStream = new ByteArrayInputStream(data, offset + 1, len - 1);
             DataInput dataIn = new DataInputStream(inStream);
             try {
-                String geometry = recordToString(new ARecordSerializerDeserializer(recType).deserialize(dataIn));
-                MapOGCStructure structure = geoJsonImporter.executeOGC(0, geometry, null);
-                OGCGeometry ogcGeometry =
-                        OGCGeometry.createFromOGCStructure(structure.m_ogcStructure, structure.m_spatialReference);
-                ByteBuffer buffer = ogcGeometry.asBinary();
-                byte[] wKBGeometryBuffer = buffer.array();
+                String geometryGeoJSON = recordToString(new ARecordSerializerDeserializer(recType).deserialize(dataIn));
+                Geometry geometry = objectMapper.readValue(geometryGeoJSON, Geometry.class);
                 out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
-                out.writeInt(wKBGeometryBuffer.length);
-                out.write(wKBGeometryBuffer);
+                AGeometrySerializerDeserializer.INSTANCE.serialize(geometry, out);
                 result.set(resultStorage);
             } catch (IOException e) {
                 throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAreaDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAreaDescriptor.java
index a8d983c..c10bbf2 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAreaDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAreaDescriptor.java
@@ -21,12 +21,10 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.Geometry;
-import com.esri.core.geometry.GeometryCursor;
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public class STAreaDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -39,18 +37,15 @@
     };
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        double area;
-        if (!"GeometryCollection".equals(geometry.geometryType())) {
-            area = geometry.getEsriGeometry().calculateArea2D();
-        } else {
-            GeometryCursor cursor = geometry.getEsriGeometryCursor();
-            Geometry geometry1 = cursor.next();
-            area = 0;
-            while (geometry1 != null) {
-                area += geometry1.calculateArea2D();
-                geometry1 = cursor.next();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        double area = 0;
+        if (StringUtils.equals(geometry.getGeometryType(), Geometry.TYPENAME_GEOMETRYCOLLECTION)) {
+            for (int i = 0; i < geometry.getNumGeometries(); i++) {
+                Geometry subGeom = geometry.getGeometryN(i);
+                area += subGeom.getArea();
             }
+        } else {
+            area = geometry.getArea();
         }
         return area;
     }
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsBinaryDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsBinaryDescriptor.java
index e0e43bb..9ee77c1 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsBinaryDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsBinaryDescriptor.java
@@ -18,12 +18,13 @@
  */
 package org.apache.asterix.geo.evaluators.functions;
 
+import org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.GeoFunctionUtils;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.io.WKBWriter;
 
 public class STAsBinaryDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -31,8 +32,10 @@
     public static final IFunctionDescriptorFactory FACTORY = STAsBinaryDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        return geometry.asBinary().array();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        WKBWriter wkbWriter = new WKBWriter(GeoFunctionUtils.getCoordinateDimension(geometry),
+                GeoFunctionUtils.LITTLE_ENDIAN_BYTEORDER);
+        return wkbWriter.write(geometry);
     }
 
     @Override
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsGeoJSONDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsGeoJSONDescriptor.java
index 760261d..f5d2217 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsGeoJSONDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsGeoJSONDescriptor.java
@@ -18,21 +18,41 @@
  */
 package org.apache.asterix.geo.evaluators.functions;
 
+import org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.JtsModule;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.locationtech.jts.geom.Geometry;
 
-import com.esri.core.geometry.ogc.OGCGeometry;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
 
 public class STAsGeoJSONDescriptor extends AbstractSTSingleGeometryDescriptor {
 
     private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = STAsGeoJSONDescriptor::new;
+    private static final Logger LOGGER = LogManager.getLogger();
+    private final ObjectMapper mapper;
+
+    public STAsGeoJSONDescriptor() {
+        mapper = new ObjectMapper();
+    }
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        return geometry.asGeoJson();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        String geoJson = null;
+        try {
+            mapper.registerModule(new JtsModule());
+            geoJson = mapper.writeValueAsString(geometry);
+        } catch (JsonProcessingException e) {
+            LOGGER.debug("JSON Processing exception during STAsGeoJSON function");
+            throw HyracksDataException.create(ErrorCode.PARSING_ERROR);
+        }
+        return geoJson;
     }
 
     @Override
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsTextDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsTextDescriptor.java
index 12bdb30..372c73e 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsTextDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsTextDescriptor.java
@@ -18,12 +18,13 @@
  */
 package org.apache.asterix.geo.evaluators.functions;
 
+import org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.GeoFunctionUtils;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.io.WKTWriter;
 
 public class STAsTextDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -31,8 +32,13 @@
     public static final IFunctionDescriptorFactory FACTORY = STAsTextDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        return geometry.asText();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        try {
+            WKTWriter wktWriter = new WKTWriter(GeoFunctionUtils.getCoordinateDimension(geometry));
+            return wktWriter.write(geometry);
+        } catch (Exception e) {
+            throw new HyracksDataException("Failed to convert geometry to WKT: " + e.getMessage(), e);
+        }
     }
 
     @Override
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STBoundaryDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STBoundaryDescriptor.java
index 9bd6ba1..5d6abaf 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STBoundaryDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STBoundaryDescriptor.java
@@ -22,8 +22,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public class STBoundaryDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -31,8 +30,8 @@
     public static final IFunctionDescriptorFactory FACTORY = STBoundaryDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        return geometry.boundary();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        return geometry.getBoundary();
     }
 
     @Override
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STContainsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STContainsDescriptor.java
index 34c21bf..e0461ff 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STContainsDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STContainsDescriptor.java
@@ -22,8 +22,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public class STContainsDescriptor extends AbstractSTDoubleGeometryDescriptor {
 
@@ -31,7 +30,7 @@
     public static final IFunctionDescriptorFactory FACTORY = STContainsDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+    protected Object evaluateOGCGeometry(Geometry geometry0, Geometry geometry1) throws HyracksDataException {
         return geometry0.contains(geometry1);
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCoordDimDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCoordDimDescriptor.java
index a0c10f7..da8ee66 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCoordDimDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCoordDimDescriptor.java
@@ -18,12 +18,12 @@
  */
 package org.apache.asterix.geo.evaluators.functions;
 
+import org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.GeoFunctionUtils;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public class STCoordDimDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -31,8 +31,8 @@
     public static final IFunctionDescriptorFactory FACTORY = STCoordDimDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        return geometry.coordinateDimension();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        return GeoFunctionUtils.getCoordinateDimension(geometry);
     }
 
     @Override
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCrossesDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCrossesDescriptor.java
index c276e7e..f868354 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCrossesDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCrossesDescriptor.java
@@ -22,8 +22,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public class STCrossesDescriptor extends AbstractSTDoubleGeometryDescriptor {
 
@@ -31,7 +30,7 @@
     public static final IFunctionDescriptorFactory FACTORY = STCrossesDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+    protected Object evaluateOGCGeometry(Geometry geometry0, Geometry geometry1) throws HyracksDataException {
         return geometry0.crosses(geometry1);
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDifferenceDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDifferenceDescriptor.java
index 8a49d4b..3ea165f 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDifferenceDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDifferenceDescriptor.java
@@ -22,8 +22,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public class STDifferenceDescriptor extends AbstractSTDoubleGeometryDescriptor {
 
@@ -31,7 +30,7 @@
     public static final IFunctionDescriptorFactory FACTORY = STDifferenceDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+    protected Object evaluateOGCGeometry(Geometry geometry0, Geometry geometry1) throws HyracksDataException {
         return geometry0.difference(geometry1);
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDimensionDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDimensionDescriptor.java
index 34d7282..38683a7 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDimensionDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDimensionDescriptor.java
@@ -22,8 +22,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public class STDimensionDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -31,8 +30,8 @@
     public static final IFunctionDescriptorFactory FACTORY = STDimensionDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        return geometry.dimension();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        return geometry.getDimension();
     }
 
     @Override
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDisjointDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDisjointDescriptor.java
index 0a890b9..5317f49 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDisjointDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDisjointDescriptor.java
@@ -22,8 +22,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public class STDisjointDescriptor extends AbstractSTDoubleGeometryDescriptor {
 
@@ -31,7 +30,7 @@
     public static final IFunctionDescriptorFactory FACTORY = STDisjointDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+    protected Object evaluateOGCGeometry(Geometry geometry0, Geometry geometry1) throws HyracksDataException {
         return geometry0.disjoint(geometry1);
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDistanceDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDistanceDescriptor.java
index 75b8b41..b68a6af 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDistanceDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDistanceDescriptor.java
@@ -22,8 +22,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public class STDistanceDescriptor extends AbstractSTDoubleGeometryDescriptor {
 
@@ -31,7 +30,7 @@
     public static final IFunctionDescriptorFactory FACTORY = STDistanceDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+    protected Object evaluateOGCGeometry(Geometry geometry0, Geometry geometry1) throws HyracksDataException {
         return geometry0.distance(geometry1);
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEndPointDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEndPointDescriptor.java
index ab0ba5c..36bc258 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEndPointDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEndPointDescriptor.java
@@ -20,11 +20,11 @@
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCCurve;
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.LineString;
 
 public class STEndPointDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -32,12 +32,12 @@
     public static final IFunctionDescriptorFactory FACTORY = STEndPointDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        if (geometry instanceof OGCCurve) {
-            return ((OGCCurve) geometry).endPoint();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        if (StringUtils.equals(geometry.getGeometryType(), Geometry.TYPENAME_LINESTRING)) {
+            return ((LineString) geometry).getEndPoint();
         } else {
-            throw new UnsupportedOperationException(
-                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+            throw new UnsupportedOperationException("The operation " + getIdentifier()
+                    + " is not supported for the type " + geometry.getGeometryType());
         }
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEnvelopeDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEnvelopeDescriptor.java
index 13555c6..2ca68fd 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEnvelopeDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEnvelopeDescriptor.java
@@ -22,8 +22,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public class STEnvelopeDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -31,8 +30,8 @@
     public static final IFunctionDescriptorFactory FACTORY = STEnvelopeDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        return geometry.envelope();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        return geometry.getEnvelope();
     }
 
     @Override
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEqualsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEqualsDescriptor.java
index e0c0cf8..c40ee62 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEqualsDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEqualsDescriptor.java
@@ -22,8 +22,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public class STEqualsDescriptor extends AbstractSTDoubleGeometryDescriptor {
 
@@ -31,7 +30,7 @@
     public static final IFunctionDescriptorFactory FACTORY = STEqualsDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+    protected Object evaluateOGCGeometry(Geometry geometry0, Geometry geometry1) throws HyracksDataException {
         return geometry0.equals(geometry1);
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STExteriorRingDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STExteriorRingDescriptor.java
index 5888b63..71cb2b7 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STExteriorRingDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STExteriorRingDescriptor.java
@@ -20,11 +20,11 @@
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
-import com.esri.core.geometry.ogc.OGCPolygon;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.Polygon;
 
 public class STExteriorRingDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -32,12 +32,12 @@
     public static final IFunctionDescriptorFactory FACTORY = STExteriorRingDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        if (geometry instanceof OGCPolygon) {
-            return ((OGCPolygon) geometry).exteriorRing();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        if (StringUtils.equals(geometry.getGeometryType(), Geometry.TYPENAME_POLYGON)) {
+            return ((Polygon) geometry).getExteriorRing();
         } else {
-            throw new UnsupportedOperationException(
-                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+            throw new UnsupportedOperationException("The operation " + getIdentifier()
+                    + " is not supported for the type " + geometry.getGeometryType());
         }
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextDescriptor.java
index 73272ae..fc097ce 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextDescriptor.java
@@ -22,8 +22,8 @@
 import java.io.DataInputStream;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.nio.ByteBuffer;
 
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
@@ -40,12 +40,9 @@
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-
-import com.esri.core.geometry.OGCStructure;
-import com.esri.core.geometry.OperatorImportFromWkt;
-import com.esri.core.geometry.SpatialReference;
-import com.esri.core.geometry.WktImportFlags;
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.io.ParseException;
+import org.locationtech.jts.io.WKTReader;
 
 public class STGeomFromTextDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
@@ -77,7 +74,6 @@
         private DataOutput out;
         private IPointable inputArg;
         private IScalarEvaluator eval;
-        private OperatorImportFromWkt wktImporter;
 
         public STGeomFromTextEvaluator(IScalarEvaluatorFactory[] args, IEvaluatorContext ctx)
                 throws HyracksDataException {
@@ -85,7 +81,6 @@
             out = resultStorage.getDataOutput();
             inputArg = new VoidPointable();
             eval = args[0].createScalarEvaluator(ctx);
-            wktImporter = OperatorImportFromWkt.local();
         }
 
         @Override
@@ -102,19 +97,13 @@
             ByteArrayInputStream inStream = new ByteArrayInputStream(data, offset + 1, len - 1);
             DataInputStream dataIn = new DataInputStream(inStream);
             try {
-                String geometry = AStringSerializerDeserializer.INSTANCE.deserialize(dataIn).getStringValue();
-                OGCStructure structure;
-
-                structure = wktImporter.executeOGC(WktImportFlags.wktImportNonTrusted, geometry, null);
-                OGCGeometry ogcGeometry = OGCGeometry.createFromOGCStructure(structure, SpatialReference.create(4326));
-                ByteBuffer buffer = ogcGeometry.asBinary();
-                byte[] wKBGeometryBuffer = buffer.array();
+                WKTReader reader = new WKTReader();
+                String wktString = AStringSerializerDeserializer.INSTANCE.deserialize(dataIn).getStringValue();
+                Geometry geometry = reader.read(wktString);
                 out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
-                out.writeInt(wKBGeometryBuffer.length);
-                out.write(wKBGeometryBuffer);
+                AGeometrySerializerDeserializer.INSTANCE.serialize(geometry, out);
                 result.set(resultStorage);
-
-            } catch (IOException e) {
+            } catch (ParseException | IOException e) {
                 throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                         ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
             }
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextSRIDDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextSRIDDescriptor.java
index aca726e..3a3dddb 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextSRIDDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextSRIDDescriptor.java
@@ -22,9 +22,8 @@
 import java.io.DataInputStream;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.nio.ByteBuffer;
 
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
@@ -41,12 +40,9 @@
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-
-import com.esri.core.geometry.OGCStructure;
-import com.esri.core.geometry.OperatorImportFromWkt;
-import com.esri.core.geometry.SpatialReference;
-import com.esri.core.geometry.WktImportFlags;
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.io.ParseException;
+import org.locationtech.jts.io.WKTReader;
 
 public class STGeomFromTextSRIDDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
@@ -80,7 +76,6 @@
         private IScalarEvaluator eval;
         private IPointable inputArg0;
         private IScalarEvaluator eval0;
-        private OperatorImportFromWkt wktImporter;
 
         public STGeomFromTextSRIDEvaluator(IScalarEvaluatorFactory[] args, IEvaluatorContext ctx)
                 throws HyracksDataException {
@@ -90,7 +85,6 @@
             eval = args[0].createScalarEvaluator(ctx);
             inputArg0 = new VoidPointable();
             eval0 = args[1].createScalarEvaluator(ctx);
-            wktImporter = OperatorImportFromWkt.local();
         }
 
         @Override
@@ -116,20 +110,13 @@
             ByteArrayInputStream inStream = new ByteArrayInputStream(data, offset + 1, len - 1);
             DataInputStream dataIn = new DataInputStream(inStream);
             try {
-                String geometry = AStringSerializerDeserializer.INSTANCE.deserialize(dataIn).getStringValue();
-                int srid = (int) AInt64SerializerDeserializer.getLong(data0, offset0 + 1);
-                OGCStructure structure;
-
-                structure = wktImporter.executeOGC(WktImportFlags.wktImportNonTrusted, geometry, null);
-                OGCGeometry ogcGeometry = OGCGeometry.createFromOGCStructure(structure, SpatialReference.create(srid));
-                ByteBuffer buffer = ogcGeometry.asBinary();
-                byte[] wKBGeometryBuffer = buffer.array();
+                WKTReader reader = new WKTReader();
+                String wktString = AStringSerializerDeserializer.INSTANCE.deserialize(dataIn).getStringValue();
+                Geometry geometry = reader.read(wktString);
                 out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
-                out.writeInt(wKBGeometryBuffer.length);
-                out.write(wKBGeometryBuffer);
+                AGeometrySerializerDeserializer.INSTANCE.serialize(geometry, out);
                 result.set(resultStorage);
-
-            } catch (IOException e) {
+            } catch (ParseException | IOException e) {
                 throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                         ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
             }
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomentryTypeDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomentryTypeDescriptor.java
index 8a432d1..96e5931 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomentryTypeDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomentryTypeDescriptor.java
@@ -22,8 +22,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public class STGeomentryTypeDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -31,8 +30,8 @@
     public static final IFunctionDescriptorFactory FACTORY = STGeomentryTypeDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        return geometry.geometryType();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        return geometry.getGeometryType();
     }
 
     @Override
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeometryNDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeometryNDescriptor.java
index 5d9642e..4c8cc9a 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeometryNDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeometryNDescriptor.java
@@ -20,11 +20,10 @@
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
-import com.esri.core.geometry.ogc.OGCGeometryCollection;
+import org.locationtech.jts.geom.Geometry;
 
 public class STGeometryNDescriptor extends AbstractSTGeometryNDescriptor {
 
@@ -38,12 +37,12 @@
     }
 
     @Override
-    protected OGCGeometry evaluateOGCGeometry(OGCGeometry geometry, int n) throws HyracksDataException {
-        if (geometry instanceof OGCGeometryCollection) {
-            return ((OGCGeometryCollection) geometry).geometryN(n);
+    protected Geometry evaluateOGCGeometry(Geometry geometry, int n) throws HyracksDataException {
+        if (StringUtils.equals(geometry.getGeometryType(), Geometry.TYPENAME_GEOMETRYCOLLECTION)) {
+            return geometry.getGeometryN(n);
         } else {
-            throw new UnsupportedOperationException(
-                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+            throw new UnsupportedOperationException("The operation " + getIdentifier()
+                    + " is not supported for the type " + geometry.getGeometryType());
         }
     }
 }
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STInteriorRingNDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STInteriorRingNDescriptor.java
index 5f6d956..9dc8bd9 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STInteriorRingNDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STInteriorRingNDescriptor.java
@@ -20,11 +20,11 @@
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
-import com.esri.core.geometry.ogc.OGCPolygon;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.Polygon;
 
 public class STInteriorRingNDescriptor extends AbstractSTGeometryNDescriptor {
 
@@ -38,12 +38,12 @@
     }
 
     @Override
-    protected OGCGeometry evaluateOGCGeometry(OGCGeometry geometry, int n) throws HyracksDataException {
-        if (geometry instanceof OGCPolygon) {
-            return ((OGCPolygon) geometry).interiorRingN(n);
+    protected Geometry evaluateOGCGeometry(Geometry geometry, int n) throws HyracksDataException {
+        if (StringUtils.equals(geometry.getGeometryType(), Geometry.TYPENAME_POLYGON)) {
+            return ((Polygon) geometry).getInteriorRingN(n);
         } else {
-            throw new UnsupportedOperationException(
-                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+            throw new UnsupportedOperationException("The operation " + getIdentifier()
+                    + " is not supported for the type " + geometry.getGeometryType());
         }
     }
 }
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectionDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectionDescriptor.java
index 5a3d2bf..6888b2b 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectionDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectionDescriptor.java
@@ -22,8 +22,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public class STIntersectionDescriptor extends AbstractSTDoubleGeometryDescriptor {
 
@@ -31,7 +30,7 @@
     public static final IFunctionDescriptorFactory FACTORY = STIntersectionDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+    protected Object evaluateOGCGeometry(Geometry geometry0, Geometry geometry1) throws HyracksDataException {
         return geometry0.intersection(geometry1);
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectsDescriptor.java
index 34a1407..a8a4b78 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectsDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectsDescriptor.java
@@ -22,8 +22,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public class STIntersectsDescriptor extends AbstractSTDoubleGeometryDescriptor {
 
@@ -31,7 +30,7 @@
     public static final IFunctionDescriptorFactory FACTORY = STIntersectsDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+    protected Object evaluateOGCGeometry(Geometry geometry0, Geometry geometry1) throws HyracksDataException {
         return geometry0.intersects(geometry1);
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsClosedDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsClosedDescriptor.java
index ff55f9f..cedda05 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsClosedDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsClosedDescriptor.java
@@ -22,15 +22,10 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCCurve;
-import com.esri.core.geometry.ogc.OGCGeometry;
-import com.esri.core.geometry.ogc.OGCGeometryCollection;
-import com.esri.core.geometry.ogc.OGCMultiCurve;
-import com.esri.core.geometry.ogc.OGCMultiPoint;
-import com.esri.core.geometry.ogc.OGCMultiPolygon;
-import com.esri.core.geometry.ogc.OGCPoint;
-import com.esri.core.geometry.ogc.OGCPolygon;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryCollection;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.MultiLineString;
 
 public class STIsClosedDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -38,30 +33,36 @@
     public static final IFunctionDescriptorFactory FACTORY = STIsClosedDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
         return isClosed(geometry);
     }
 
-    private boolean isClosed(OGCGeometry geometry) {
-        if (geometry instanceof OGCCurve) {
-            return ((OGCCurve) geometry).isClosed();
-        } else if (geometry instanceof OGCMultiCurve) {
-            return ((OGCMultiCurve) geometry).isClosed();
-        } else if (geometry instanceof OGCMultiPoint || geometry instanceof OGCMultiPolygon
-                || geometry instanceof OGCPolygon || geometry instanceof OGCPoint) {
-            return true;
-        } else if (geometry instanceof OGCGeometryCollection) {
-            OGCGeometryCollection geometryCollection = (OGCGeometryCollection) geometry;
-            int num = geometryCollection.numGeometries();
-            for (int i = 0; i < num; ++i) {
-                if (!isClosed(geometryCollection.geometryN(i))) {
-                    return false;
+    private boolean isClosed(Geometry geometry) {
+        switch (geometry.getGeometryType()) {
+            case Geometry.TYPENAME_LINESTRING:
+                return ((LineString) geometry).isClosed();
+
+            case Geometry.TYPENAME_MULTILINESTRING:
+                return ((MultiLineString) geometry).isClosed();
+
+            case Geometry.TYPENAME_MULTIPOINT:
+            case Geometry.TYPENAME_MULTIPOLYGON:
+            case Geometry.TYPENAME_POLYGON:
+            case Geometry.TYPENAME_POINT:
+                return true;
+
+            case Geometry.TYPENAME_GEOMETRYCOLLECTION:
+                GeometryCollection geometryCollection = (GeometryCollection) geometry;
+                int num = geometryCollection.getNumGeometries();
+                for (int i = 0; i < num; ++i) {
+                    if (!isClosed(geometryCollection.getGeometryN(i))) {
+                        return false;
+                    }
                 }
-            }
-            return true;
-        } else {
-            throw new UnsupportedOperationException(
-                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+                return true;
+
+            default:
+                throw new IllegalArgumentException("Unsupported geometry type: " + geometry.getGeometryType());
         }
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsCollectionDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsCollectionDescriptor.java
index 544fdf3..f6a8578 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsCollectionDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsCollectionDescriptor.java
@@ -22,8 +22,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public class STIsCollectionDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -31,8 +30,8 @@
     public static final IFunctionDescriptorFactory FACTORY = STIsCollectionDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        String type = geometry.geometryType();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        String type = geometry.getGeometryType();
         return "GeometryCollection".equals(type) || "MultiLineString".equals(type) || "MultiPoint".equals(type)
                 || "MultiPolygon".equals(type);
     }
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsEmptyDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsEmptyDescriptor.java
index 1adcb0e..569806c 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsEmptyDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsEmptyDescriptor.java
@@ -22,8 +22,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public class STIsEmptyDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -31,7 +30,7 @@
     public static final IFunctionDescriptorFactory FACTORY = STIsEmptyDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
         return geometry.isEmpty();
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsRingDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsRingDescriptor.java
index 02ea049a..0c84c1c 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsRingDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsRingDescriptor.java
@@ -20,11 +20,11 @@
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCCurve;
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.LineString;
 
 public class STIsRingDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -32,12 +32,12 @@
     public static final IFunctionDescriptorFactory FACTORY = STIsRingDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        if (geometry instanceof OGCCurve) {
-            return ((OGCCurve) geometry).isRing();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        if (StringUtils.equals(geometry.getGeometryType(), Geometry.TYPENAME_LINESTRING)) {
+            return ((LineString) geometry).isRing();
         } else {
-            throw new UnsupportedOperationException(
-                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+            throw new UnsupportedOperationException("The operation " + getIdentifier()
+                    + " is not supported for the type " + geometry.getGeometryType());
 
         }
     }
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsSimpleDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsSimpleDescriptor.java
index b8b2d63..2b6c468 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsSimpleDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsSimpleDescriptor.java
@@ -22,8 +22,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public class STIsSimpleDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -31,7 +30,7 @@
     public static final IFunctionDescriptorFactory FACTORY = STIsSimpleDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
         return geometry.isSimple();
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLengthDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLengthDescriptor.java
index 7357813..8909ed3 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLengthDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLengthDescriptor.java
@@ -20,14 +20,12 @@
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.Geometry;
-import com.esri.core.geometry.GeometryCursor;
-import com.esri.core.geometry.ogc.OGCGeometry;
-import com.esri.core.geometry.ogc.OGCLineString;
-import com.esri.core.geometry.ogc.OGCMultiLineString;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.MultiLineString;
 
 public class STLengthDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -35,21 +33,20 @@
     public static final IFunctionDescriptorFactory FACTORY = STLengthDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        if (geometry instanceof OGCLineString) {
-            return geometry.getEsriGeometry().calculateLength2D();
-        } else if (geometry instanceof OGCMultiLineString) {
-            GeometryCursor cursor = geometry.getEsriGeometryCursor();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        if (StringUtils.equals(geometry.getGeometryType(), Geometry.TYPENAME_LINESTRING)) {
+            return geometry.getLength();
+        } else if (StringUtils.equals(geometry.getGeometryType(), Geometry.TYPENAME_MULTILINESTRING)) {
             double length = 0;
-            Geometry geometry1 = cursor.next();
-            while (geometry1 != null) {
-                length += geometry1.calculateLength2D();
-                geometry1 = cursor.next();
+            MultiLineString multiLine = (MultiLineString) geometry;
+            for (int i = 0; i < multiLine.getNumGeometries(); i++) {
+                LineString lineString = (LineString) multiLine.getGeometryN(i);
+                length += lineString.getLength();
             }
             return length;
         } else {
-            throw new UnsupportedOperationException(
-                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+            throw new UnsupportedOperationException("The operation " + getIdentifier()
+                    + " is not supported for the type " + geometry.getGeometryType());
         }
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLineFromMultiPointDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLineFromMultiPointDescriptor.java
index b716b5c..82e1c3d 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLineFromMultiPointDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLineFromMultiPointDescriptor.java
@@ -20,35 +20,42 @@
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.Point;
-import com.esri.core.geometry.Polyline;
-import com.esri.core.geometry.SpatialReference;
-import com.esri.core.geometry.ogc.OGCGeometry;
-import com.esri.core.geometry.ogc.OGCLineString;
-import com.esri.core.geometry.ogc.OGCMultiPoint;
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.MultiPoint;
 
 public class STLineFromMultiPointDescriptor extends AbstractSTSingleGeometryDescriptor {
 
     private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = STLineFromMultiPointDescriptor::new;
+    private final GeometryFactory geometryFactory;
+
+    public STLineFromMultiPointDescriptor() {
+        geometryFactory = new GeometryFactory();
+    }
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        if (geometry instanceof OGCMultiPoint) {
-            Polyline polyline = new Polyline();
-            OGCMultiPoint multiPoint = (OGCMultiPoint) geometry;
-            int numPoints = multiPoint.numGeometries();
-            polyline.startPath((Point) multiPoint.geometryN(0).getEsriGeometry());
-            for (int i = 1; i < numPoints; i++) {
-                polyline.lineTo((Point) multiPoint.geometryN(i).getEsriGeometry());
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        if (StringUtils.equals(geometry.getGeometryType(), Geometry.TYPENAME_MULTIPOINT)) {
+            MultiPoint multiPoint = (MultiPoint) geometry;
+            int numPoints = multiPoint.getNumGeometries();
+
+            if (numPoints < 2) {
+                throw new UnsupportedOperationException("A LineString requires at least two points.");
             }
-            return new OGCLineString(polyline, 0, SpatialReference.create(4326));
+
+            Coordinate[] coordinates = new Coordinate[numPoints];
+            for (int i = 0; i < numPoints; i++) {
+                coordinates[i] = multiPoint.getGeometryN(i).getCoordinate();
+            }
+            return geometryFactory.createLineString(coordinates);
         } else {
-            throw new UnsupportedOperationException(
-                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+            throw new UnsupportedOperationException("The operation " + getIdentifier()
+                    + " is not supported for the type " + geometry.getGeometryType());
         }
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMBRDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMBRDescriptor.java
index 55e358c..6353918 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMBRDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMBRDescriptor.java
@@ -25,9 +25,8 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.Envelope;
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Envelope;
+import org.locationtech.jts.geom.Geometry;
 
 public class STMBRDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -41,14 +40,12 @@
     };
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
         AMutableRectangle aRectangle = new AMutableRectangle(null, null);
         AMutablePoint[] aPoint = { new AMutablePoint(0, 0), new AMutablePoint(0, 0) };
-        Envelope env = new Envelope();
-        geometry.getEsriGeometry().queryEnvelope(env);
-        aPoint[0].setValue(env.getXMin(), env.getYMin());
-        aPoint[1].setValue(env.getXMax(), env.getYMax());
+        Envelope envelope = geometry.getEnvelopeInternal();
+        aPoint[0].setValue(envelope.getMinX(), envelope.getMinY());
+        aPoint[1].setValue(envelope.getMaxX(), envelope.getMaxY());
         aRectangle.setValue(aPoint[0], aPoint[1]);
         return aRectangle;
     }
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMBREnlargeDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMBREnlargeDescriptor.java
index b3bfb33..50ddd28 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMBREnlargeDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMBREnlargeDescriptor.java
@@ -46,9 +46,9 @@
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.data.std.util.ByteArrayAccessibleInputStream;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-
-import com.esri.core.geometry.Envelope;
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Envelope;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
 
 public class STMBREnlargeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
@@ -61,6 +61,12 @@
         }
     };
 
+    private final GeometryFactory geometryFactory;
+
+    public STMBREnlargeDescriptor() {
+        geometryFactory = new GeometryFactory();
+    }
+
     @Override
     public FunctionIdentifier getIdentifier() {
         return BuiltinFunctions.ST_MBR_ENLARGE;
@@ -114,14 +120,13 @@
                         }
 
                         inStream.setContent(data0, offset0 + 1, len - 1);
-                        OGCGeometry geometry =
-                                AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn).getGeometry();
-                        geometry.getEsriGeometry().queryEnvelope(env);
+                        Geometry geometry = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn).getGeometry();
+                        Envelope env = geometry.getEnvelopeInternal();
                         double expandValue =
                                 ATypeHierarchy.getDoubleValue(getIdentifier().getName(), 0, data1, offset1);
                         AMutableRectangle expandedMBR = new AMutableRectangle(
-                                new AMutablePoint(env.getXMin() - expandValue, env.getYMin() - expandValue),
-                                new AMutablePoint(env.getXMax() + expandValue, env.getYMax() + expandValue));
+                                new AMutablePoint(env.getMinX() - expandValue, env.getMinY() - expandValue),
+                                new AMutablePoint(env.getMaxX() + expandValue, env.getMaxY() + expandValue));
                         rectangleSerde.serialize(expandedMBR, out);
                         result.set(resultStorage);
                     }
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMDescriptor.java
index e208771..71fcacf 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMDescriptor.java
@@ -20,24 +20,36 @@
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
-import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.CoordinateXYZM;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.Point;
 
 public class STMDescriptor extends AbstractSTSingleGeometryDescriptor {
 
     private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = STMDescriptor::new;
+    private static final Logger LOGGER = LogManager.getLogger();
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        if (geometry instanceof OGCPoint) {
-            return ((OGCPoint) geometry).M();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        if (StringUtils.equals(geometry.getGeometryType(), Geometry.TYPENAME_POINT)) {
+            Point point = (Point) geometry;
+            Coordinate coordinate = point.getCoordinate();
+            if (coordinate instanceof CoordinateXYZM) {
+                return coordinate.getM();
+            } else {
+                LOGGER.debug("The provided point does not have an M value.");
+                return Double.NaN;
+            }
         } else {
-            throw new UnsupportedOperationException(
-                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+            throw new UnsupportedOperationException("The operation " + getIdentifier()
+                    + " is not supported for the type " + geometry.getGeometryType());
         }
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakeEnvelopeDescriptorSRID.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakeEnvelopeDescriptorSRID.java
index 3aa9e7b..f8cc270 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakeEnvelopeDescriptorSRID.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakeEnvelopeDescriptorSRID.java
@@ -20,8 +20,8 @@
 
 import java.io.DataOutput;
 import java.io.IOException;
-import java.nio.ByteBuffer;
 
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.types.ATypeTag;
@@ -35,10 +35,9 @@
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-
-import com.esri.core.geometry.Envelope;
-import com.esri.core.geometry.SpatialReference;
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.Polygon;
 
 public class STMakeEnvelopeDescriptorSRID extends AbstractGetValDescriptor {
 
@@ -70,14 +69,19 @@
         private DataOutput out;
         private IPointable inputArg0;
         private IScalarEvaluator eval0;
+        private Coordinate coordinate0;
         private IPointable inputArg1;
         private IScalarEvaluator eval1;
+        private Coordinate coordinate1;
         private IPointable inputArg2;
         private IScalarEvaluator eval2;
+        private Coordinate coordinate2;
         private IPointable inputArg3;
         private IScalarEvaluator eval3;
+        private Coordinate coordinate3;
         private IPointable inputArg4;
         private IScalarEvaluator eval4;
+        private final GeometryFactory geometryFactory;
 
         public STMakeEnvelopeEvaluator(IScalarEvaluatorFactory[] args, IEvaluatorContext ctx)
                 throws HyracksDataException {
@@ -93,6 +97,11 @@
             eval3 = args[3].createScalarEvaluator(ctx);
             inputArg4 = new VoidPointable();
             eval4 = args[4].createScalarEvaluator(ctx);
+            geometryFactory = new GeometryFactory();
+            coordinate0 = new Coordinate();
+            coordinate1 = new Coordinate();
+            coordinate2 = new Coordinate();
+            coordinate3 = new Coordinate();
         }
 
         @Override
@@ -116,23 +125,30 @@
             byte[] data3 = inputArg3.getByteArray();
             int offset3 = inputArg3.getStartOffset();
 
+            //Spatial Reference System Identifier (SRID), currently not used.
             eval4.evaluate(tuple, inputArg4);
             byte[] data4 = inputArg4.getByteArray();
             int offset4 = inputArg4.getStartOffset();
 
             try {
+                double xmin = getVal(data0, offset0);
+                double ymin = getVal(data1, offset1);
+                double xmax = getVal(data2, offset2);
+                double ymax = getVal(data3, offset3);
+                coordinate0.setX(xmin);
+                coordinate0.setY(ymin);
+                coordinate1.setX(xmin);
+                coordinate1.setY(ymax);
+                coordinate2.setX(xmax);
+                coordinate2.setY(ymax);
+                coordinate3.setX(xmax);
+                coordinate3.setY(ymin);
 
-                OGCGeometry ogcGeometry =
-                        OGCGeometry
-                                .createFromEsriGeometry(
-                                        new Envelope(getVal(data0, offset0), getVal(data1, offset1),
-                                                getVal(data2, offset2), getVal(data3, offset3)),
-                                        SpatialReference.create((int) getVal(data4, offset4)));
-                ByteBuffer buffer = ogcGeometry.asBinary();
-                byte[] bytes = buffer.array();
+                Coordinate[] coords =
+                        new Coordinate[] { coordinate0, coordinate1, coordinate2, coordinate3, coordinate0 };
+                Polygon polygon = geometryFactory.createPolygon(coords);
                 out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
-                out.writeInt(bytes.length);
-                out.write(bytes);
+                AGeometrySerializerDeserializer.INSTANCE.serialize(polygon, out);
                 result.set(resultStorage);
             } catch (IOException e) {
                 throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DDescriptor.java
index 4a5cf7e..8009510 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DDescriptor.java
@@ -22,7 +22,6 @@
 import java.io.IOException;
 
 import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
-import org.apache.asterix.om.base.AGeometry;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.types.ATypeTag;
@@ -35,10 +34,9 @@
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-
-import com.esri.core.geometry.Point;
-import com.esri.core.geometry.SpatialReference;
-import com.esri.core.geometry.ogc.OGCPoint;
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.Point;
 
 public class STMakePoint3DDescriptor extends AbstractGetValDescriptor {
 
@@ -73,7 +71,7 @@
         private final IScalarEvaluator eval1;
         private final IScalarEvaluator eval2;
         private Point point;
-        private AGeometry pointGeometry;
+        private final GeometryFactory geometryFactory;
 
         public STMakePoint3DEvaluator(IScalarEvaluatorFactory[] args, IEvaluatorContext ctx)
                 throws HyracksDataException {
@@ -85,8 +83,7 @@
             eval0 = args[0].createScalarEvaluator(ctx);
             eval1 = args[1].createScalarEvaluator(ctx);
             eval2 = args[2].createScalarEvaluator(ctx);
-            point = new Point(0, 0, 0);
-            pointGeometry = new AGeometry(new OGCPoint(point, SpatialReference.create(4326)));
+            geometryFactory = new GeometryFactory();
         }
 
         @Override
@@ -105,10 +102,10 @@
             resultStorage.reset();
             try {
                 out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
-                point.setX(getVal(bytes0, offset0));
-                point.setY(getVal(bytes1, offset1));
-                point.setZ(getVal(bytes2, offset2));
-                AGeometrySerializerDeserializer.INSTANCE.serialize(pointGeometry, out);
+                Coordinate coordinate =
+                        new Coordinate(getVal(bytes0, offset0), getVal(bytes1, offset1), getVal(bytes2, offset2));
+                point = geometryFactory.createPoint(coordinate);
+                AGeometrySerializerDeserializer.INSTANCE.serialize(point, out);
             } catch (IOException e1) {
                 throw HyracksDataException.create(e1);
             }
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DWithMDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DWithMDescriptor.java
index 55baa37..edc0cdc 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DWithMDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DWithMDescriptor.java
@@ -22,7 +22,6 @@
 import java.io.IOException;
 
 import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
-import org.apache.asterix.om.base.AGeometry;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.types.ATypeTag;
@@ -35,11 +34,15 @@
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import org.locationtech.jts.geom.CoordinateXYZM;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.Point;
 
-import com.esri.core.geometry.Point;
-import com.esri.core.geometry.SpatialReference;
-import com.esri.core.geometry.ogc.OGCPoint;
-
+/**
+ * TODO: Support writing geometry with 4 dimensions
+ * JTS WKBWriter currently does not support writing 4 dimensions and loses the detail.
+ * See https://github.com/locationtech/jts/issues/733 for more details.
+ */
 public class STMakePoint3DWithMDescriptor extends AbstractGetValDescriptor {
 
     private static final long serialVersionUID = 1L;
@@ -75,7 +78,7 @@
         private final IScalarEvaluator eval2;
         private final IScalarEvaluator eval3;
         private Point point;
-        private AGeometry pointGeometry;
+        private final GeometryFactory geometryFactory;
 
         public STMakePoint3DWithMEvaluator(IScalarEvaluatorFactory[] args, IEvaluatorContext ctx)
                 throws HyracksDataException {
@@ -89,8 +92,7 @@
             eval1 = args[1].createScalarEvaluator(ctx);
             eval2 = args[2].createScalarEvaluator(ctx);
             eval3 = args[3].createScalarEvaluator(ctx);
-            point = new Point(0, 0, 0);
-            pointGeometry = new AGeometry(new OGCPoint(point, SpatialReference.create(4326)));
+            geometryFactory = new GeometryFactory();
         }
 
         @Override
@@ -112,11 +114,10 @@
             resultStorage.reset();
             try {
                 out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
-                point.setX(getVal(bytes0, offset0));
-                point.setY(getVal(bytes1, offset1));
-                point.setZ(getVal(bytes2, offset2));
-                point.setM(getVal(bytes3, offset3));
-                AGeometrySerializerDeserializer.INSTANCE.serialize(pointGeometry, out);
+                CoordinateXYZM coordinate = new CoordinateXYZM(getVal(bytes0, offset0), getVal(bytes1, offset1),
+                        getVal(bytes2, offset2), getVal(bytes3, offset3));
+                point = geometryFactory.createPoint(coordinate);
+                AGeometrySerializerDeserializer.INSTANCE.serialize(point, out);
             } catch (IOException e1) {
                 throw HyracksDataException.create(e1);
             }
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePointDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePointDescriptor.java
index c3b7b36..f21b381 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePointDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePointDescriptor.java
@@ -22,7 +22,6 @@
 import java.io.IOException;
 
 import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
-import org.apache.asterix.om.base.AGeometry;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.types.ATypeTag;
@@ -35,10 +34,9 @@
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-
-import com.esri.core.geometry.Point;
-import com.esri.core.geometry.SpatialReference;
-import com.esri.core.geometry.ogc.OGCPoint;
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.Point;
 
 public class STMakePointDescriptor extends AbstractGetValDescriptor {
 
@@ -71,7 +69,7 @@
         private final IScalarEvaluator eval0;
         private final IScalarEvaluator eval1;
         private Point point;
-        private AGeometry pointGeometry;
+        private final GeometryFactory geometryFactory;
 
         public STMakePointEvaluator(IScalarEvaluatorFactory[] args, IEvaluatorContext ctx) throws HyracksDataException {
             resultStorage = new ArrayBackedValueStorage();
@@ -80,8 +78,7 @@
             inputArg1 = new VoidPointable();
             eval0 = args[0].createScalarEvaluator(ctx);
             eval1 = args[1].createScalarEvaluator(ctx);
-            point = new Point(0, 0);
-            pointGeometry = new AGeometry(new OGCPoint(point, SpatialReference.create(4326)));
+            geometryFactory = new GeometryFactory();
         }
 
         @Override
@@ -97,9 +94,9 @@
             resultStorage.reset();
             try {
                 out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
-                point.setX(getVal(bytes0, offset0));
-                point.setY(getVal(bytes1, offset1));
-                AGeometrySerializerDeserializer.INSTANCE.serialize(pointGeometry, out);
+                Coordinate coordinate = new Coordinate(getVal(bytes0, offset0), getVal(bytes1, offset1));
+                point = geometryFactory.createPoint(coordinate);
+                AGeometrySerializerDeserializer.INSTANCE.serialize(point, out);
             } catch (IOException e1) {
                 throw HyracksDataException.create(e1);
             }
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNPointsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNPointsDescriptor.java
index 309d325..c80c513 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNPointsDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNPointsDescriptor.java
@@ -20,14 +20,13 @@
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.Geometry;
-import com.esri.core.geometry.GeometryCursor;
-import com.esri.core.geometry.MultiVertexGeometry;
-import com.esri.core.geometry.Point;
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryCollection;
+import org.locationtech.jts.geom.MultiPolygon;
+import org.locationtech.jts.geom.Polygon;
 
 public class STNPointsDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -35,31 +34,45 @@
     public static final IFunctionDescriptorFactory FACTORY = STNPointsDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        Geometry esriGeometry = geometry.getEsriGeometry();
-        if (esriGeometry instanceof MultiVertexGeometry) {
-            return ((MultiVertexGeometry) esriGeometry).getPointCount();
-        } else if (esriGeometry instanceof Point) {
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        if (geometry == null) {
+            return 0;
+        }
+        if (geometry.isEmpty()) {
+            return 0;
+        }
+
+        if (StringUtils.equals(geometry.getGeometryType(), Geometry.TYPENAME_POINT))
             return 1;
-        } else if (esriGeometry == null) {
-            int count = 0;
-            GeometryCursor geometryCursor = geometry.getEsriGeometryCursor();
-            esriGeometry = geometryCursor.next();
-            while (esriGeometry != null) {
-                if (esriGeometry instanceof MultiVertexGeometry) {
-                    count += ((MultiVertexGeometry) esriGeometry).getPointCount();
-                } else if (esriGeometry instanceof Point) {
-                    count += 1;
-                }
-                esriGeometry = geometryCursor.next();
+
+        if (StringUtils.equals(geometry.getGeometryType(), Geometry.TYPENAME_POLYGON)) {
+            Polygon polygon = (Polygon) geometry;
+            int count = polygon.getExteriorRing().getCoordinates().length - 1;
+            for (int i = 0; i < polygon.getNumInteriorRing(); i++) {
+                count += polygon.getInteriorRingN(i).getCoordinates().length - 1;
             }
             return count;
-        } else if (geometry.isEmpty()) {
-            return 0;
-        } else {
-            throw new UnsupportedOperationException(
-                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
         }
+
+        if (StringUtils.equals(geometry.getGeometryType(), Geometry.TYPENAME_MULTIPOLYGON)) {
+            int count = 0;
+            MultiPolygon multiPolygon = (MultiPolygon) geometry;
+            for (int i = 0; i < multiPolygon.getNumGeometries(); i++) {
+                count += (int) evaluateOGCGeometry(multiPolygon.getGeometryN(i));
+            }
+            return count;
+        }
+
+        if (StringUtils.equals(geometry.getGeometryType(), Geometry.TYPENAME_GEOMETRYCOLLECTION)) {
+            int count = 0;
+            GeometryCollection collection = (GeometryCollection) geometry;
+            for (int i = 0; i < collection.getNumGeometries(); i++) {
+                count += (int) evaluateOGCGeometry(collection.getGeometryN(i));
+            }
+            return count;
+        }
+
+        return geometry.getCoordinates().length;
     }
 
     @Override
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNRingsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNRingsDescriptor.java
index f8e313d..ee21878 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNRingsDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNRingsDescriptor.java
@@ -20,12 +20,12 @@
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
-import com.esri.core.geometry.ogc.OGCMultiPolygon;
-import com.esri.core.geometry.ogc.OGCPolygon;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.MultiPolygon;
+import org.locationtech.jts.geom.Polygon;
 
 public class STNRingsDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -33,22 +33,22 @@
     public static final IFunctionDescriptorFactory FACTORY = STNRingsDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        if (geometry instanceof OGCPolygon) {
-            return ((OGCPolygon) geometry).numInteriorRing() + 1;
-        } else if (geometry instanceof OGCMultiPolygon) {
-            OGCMultiPolygon polygon = (OGCMultiPolygon) geometry;
-            int numGeometries = polygon.numGeometries();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        if (StringUtils.equals(geometry.getGeometryType(), Geometry.TYPENAME_POLYGON)) {
+            return ((Polygon) geometry).getNumInteriorRing() + 1;
+        } else if (StringUtils.equals(geometry.getGeometryType(), Geometry.TYPENAME_MULTIPOLYGON)) {
+            MultiPolygon polygon = (MultiPolygon) geometry;
+            int numGeometries = polygon.getNumGeometries();
             int count = 0;
             for (int i = 1; i < numGeometries + 1; i++) {
-                if (polygon.geometryN(i) instanceof OGCPolygon) {
-                    count += ((OGCPolygon) polygon.geometryN(i)).numInteriorRing() + 1;
+                if (StringUtils.equals(geometry.getGeometryType(), Geometry.TYPENAME_POLYGON)) {
+                    count += ((Polygon) polygon.getGeometryN(i)).getNumInteriorRing() + 1;
                 }
             }
             return count;
         } else {
-            throw new UnsupportedOperationException(
-                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+            throw new UnsupportedOperationException("The operation " + getIdentifier()
+                    + " is not supported for the type " + geometry.getGeometryType());
         }
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumGeometriesDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumGeometriesDescriptor.java
index e87d1f4..938f591 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumGeometriesDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumGeometriesDescriptor.java
@@ -20,11 +20,10 @@
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
-import com.esri.core.geometry.ogc.OGCGeometryCollection;
+import org.locationtech.jts.geom.Geometry;
 
 public class STNumGeometriesDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -32,9 +31,9 @@
     public static final IFunctionDescriptorFactory FACTORY = STNumGeometriesDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        if (geometry instanceof OGCGeometryCollection) {
-            return ((OGCGeometryCollection) geometry).numGeometries();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        if (StringUtils.equals(geometry.getGeometryType(), Geometry.TYPENAME_GEOMETRYCOLLECTION)) {
+            return geometry.getNumGeometries();
         } else if (!geometry.isEmpty()) {
             return 1;
         } else {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumInteriorRingsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumInteriorRingsDescriptor.java
index 19f2e11..c4f97f3 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumInteriorRingsDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumInteriorRingsDescriptor.java
@@ -20,11 +20,11 @@
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
-import com.esri.core.geometry.ogc.OGCPolygon;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.Polygon;
 
 public class STNumInteriorRingsDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -32,12 +32,12 @@
     public static final IFunctionDescriptorFactory FACTORY = STNumInteriorRingsDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        if (geometry instanceof OGCPolygon) {
-            return ((OGCPolygon) geometry).numInteriorRing();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        if (StringUtils.equals(geometry.getGeometryType(), Geometry.TYPENAME_POLYGON)) {
+            return ((Polygon) geometry).getNumInteriorRing();
         } else {
-            throw new UnsupportedOperationException(
-                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+            throw new UnsupportedOperationException("The operation " + getIdentifier()
+                    + " is not supported for the type " + geometry.getGeometryType());
         }
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STOverlapsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STOverlapsDescriptor.java
index 61286b0..509566a 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STOverlapsDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STOverlapsDescriptor.java
@@ -22,8 +22,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public class STOverlapsDescriptor extends AbstractSTDoubleGeometryDescriptor {
 
@@ -31,7 +30,7 @@
     public static final IFunctionDescriptorFactory FACTORY = STOverlapsDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+    protected Object evaluateOGCGeometry(Geometry geometry0, Geometry geometry1) throws HyracksDataException {
         return geometry0.overlaps(geometry1);
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPointNDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPointNDescriptor.java
index e0495f5..1689f9d 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPointNDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPointNDescriptor.java
@@ -20,11 +20,11 @@
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
-import com.esri.core.geometry.ogc.OGCLineString;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.LineString;
 
 public class STPointNDescriptor extends AbstractSTGeometryNDescriptor {
 
@@ -38,12 +38,12 @@
     }
 
     @Override
-    protected OGCGeometry evaluateOGCGeometry(OGCGeometry geometry, int n) throws HyracksDataException {
-        if (geometry instanceof OGCLineString) {
-            return ((OGCLineString) geometry).pointN(n);
+    protected Geometry evaluateOGCGeometry(Geometry geometry, int n) throws HyracksDataException {
+        if (StringUtils.equals(geometry.getGeometryType(), Geometry.TYPENAME_LINESTRING)) {
+            return ((LineString) geometry).getPointN(n);
         } else {
-            throw new UnsupportedOperationException(
-                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+            throw new UnsupportedOperationException("The operation " + getIdentifier()
+                    + " is not supported for the type " + geometry.getGeometryType());
         }
     }
 }
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPolygonizeDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPolygonizeDescriptor.java
index e2fcd8a..3683e1b 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPolygonizeDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPolygonizeDescriptor.java
@@ -49,11 +49,9 @@
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-
-import com.esri.core.geometry.SpatialReference;
-import com.esri.core.geometry.ogc.OGCConcreteGeometryCollection;
-import com.esri.core.geometry.ogc.OGCGeometry;
-import com.esri.core.geometry.ogc.OGCGeometryCollection;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryCollection;
+import org.locationtech.jts.geom.GeometryFactory;
 
 public class STPolygonizeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
@@ -61,6 +59,12 @@
 
     private static final long serialVersionUID = 1L;
 
+    private final GeometryFactory geometryFactory;
+
+    public STPolygonizeDescriptor() {
+        geometryFactory = new GeometryFactory();
+    }
+
     @Override
     public FunctionIdentifier getIdentifier() {
         return BuiltinFunctions.ST_POLYGONIZE;
@@ -116,13 +120,13 @@
             ByteArrayInputStream inStream = new ByteArrayInputStream(bytes, offset + 1, len - 1);
             DataInputStream dataIn = new DataInputStream(inStream);
             IACursor cursor = ((IACollection) serde.deserialize(dataIn)).getCursor();
-            List<OGCGeometry> list = new ArrayList<>();
+            List<Geometry> list = new ArrayList<>();
             while (cursor.next()) {
                 IAObject object = cursor.get();
                 list.add(((AGeometry) object).getGeometry());
             }
-            OGCGeometryCollection geometryCollection =
-                    new OGCConcreteGeometryCollection(list, SpatialReference.create(4326));
+            GeometryCollection geometryCollection =
+                    geometryFactory.createGeometryCollection(list.toArray(new Geometry[0]));
             try {
                 SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AGEOMETRY)
                         .serialize(new AGeometry(geometryCollection), out);
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STRelateDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STRelateDescriptor.java
index c3dc132..bc9c0b6 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STRelateDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STRelateDescriptor.java
@@ -44,8 +44,7 @@
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public class STRelateDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
@@ -128,9 +127,9 @@
             DataInputStream dataIn = new DataInputStream(inStream);
             String matrix = AStringSerializerDeserializer.INSTANCE.deserialize(dataIn).getStringValue();
             DataInputStream dataIn0 = new DataInputStream(new ByteArrayInputStream(bytes0, offset0 + 1, len0 - 1));
-            OGCGeometry geometry0 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn0).getGeometry();
+            Geometry geometry0 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn0).getGeometry();
             DataInputStream dataIn1 = new DataInputStream(new ByteArrayInputStream(bytes1, offset1 + 1, len1 - 1));
-            OGCGeometry geometry1 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn1).getGeometry();
+            Geometry geometry1 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn1).getGeometry();
             try {
                 boolean val = geometry0.relate(geometry1, matrix);
                 SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN)
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSRIDDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSRIDDescriptor.java
index e50678a..e19f21a 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSRIDDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSRIDDescriptor.java
@@ -22,8 +22,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public class STSRIDDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -31,8 +30,8 @@
     public static final IFunctionDescriptorFactory FACTORY = STSRIDDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        return geometry.SRID();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        return geometry.getSRID();
     }
 
     @Override
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STStartPointDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STStartPointDescriptor.java
index 127b0d5..71d3470 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STStartPointDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STStartPointDescriptor.java
@@ -20,11 +20,11 @@
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCCurve;
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.LineString;
 
 public class STStartPointDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -32,12 +32,12 @@
     public static final IFunctionDescriptorFactory FACTORY = STStartPointDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        if (geometry instanceof OGCCurve) {
-            return ((OGCCurve) geometry).startPoint();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        if (StringUtils.equals(geometry.getGeometryType(), Geometry.TYPENAME_LINESTRING)) {
+            return ((LineString) geometry).getStartPoint();
         } else {
-            throw new UnsupportedOperationException(
-                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+            throw new UnsupportedOperationException("The operation " + getIdentifier()
+                    + " is not supported for the type " + geometry.getGeometryType());
         }
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSymDifferenceDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSymDifferenceDescriptor.java
index e4d0507..b04254e 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSymDifferenceDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSymDifferenceDescriptor.java
@@ -22,8 +22,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public class STSymDifferenceDescriptor extends AbstractSTDoubleGeometryDescriptor {
 
@@ -31,7 +30,7 @@
     public static final IFunctionDescriptorFactory FACTORY = STSymDifferenceDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+    protected Object evaluateOGCGeometry(Geometry geometry0, Geometry geometry1) throws HyracksDataException {
         return geometry0.symDifference(geometry1);
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STTouchesDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STTouchesDescriptor.java
index ddf6550..0615f61 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STTouchesDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STTouchesDescriptor.java
@@ -22,8 +22,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public class STTouchesDescriptor extends AbstractSTDoubleGeometryDescriptor {
 
@@ -31,7 +30,7 @@
     public static final IFunctionDescriptorFactory FACTORY = STTouchesDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+    protected Object evaluateOGCGeometry(Geometry geometry0, Geometry geometry1) throws HyracksDataException {
         return geometry0.touches(geometry1);
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STUnionDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STUnionDescriptor.java
index 1cfea10..533a3c1 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STUnionDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STUnionDescriptor.java
@@ -22,8 +22,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public class STUnionDescriptor extends AbstractSTDoubleGeometryDescriptor {
 
@@ -31,7 +30,7 @@
     public static final IFunctionDescriptorFactory FACTORY = STUnionDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+    protected Object evaluateOGCGeometry(Geometry geometry0, Geometry geometry1) throws HyracksDataException {
         return geometry0.union(geometry1);
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STWithinDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STWithinDescriptor.java
index c78e48f..a3687ac 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STWithinDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STWithinDescriptor.java
@@ -22,8 +22,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public class STWithinDescriptor extends AbstractSTDoubleGeometryDescriptor {
 
@@ -31,7 +30,7 @@
     public static final IFunctionDescriptorFactory FACTORY = STWithinDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+    protected Object evaluateOGCGeometry(Geometry geometry0, Geometry geometry1) throws HyracksDataException {
         return geometry0.within(geometry1);
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXDescriptor.java
index 608e55cc..2bb450f 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXDescriptor.java
@@ -20,11 +20,11 @@
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
-import com.esri.core.geometry.ogc.OGCPoint;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.Point;
 
 public class STXDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -32,12 +32,12 @@
     public static final IFunctionDescriptorFactory FACTORY = STXDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        if (geometry instanceof OGCPoint) {
-            return ((OGCPoint) geometry).X();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        if (StringUtils.equals(geometry.getGeometryType(), Geometry.TYPENAME_POINT)) {
+            return ((Point) geometry).getX();
         } else {
-            throw new UnsupportedOperationException(
-                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+            throw new UnsupportedOperationException("The operation " + getIdentifier()
+                    + " is not supported for the type " + geometry.getGeometryType());
         }
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMaxDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMaxDescriptor.java
index 2431f3f..1ad10ba 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMaxDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMaxDescriptor.java
@@ -22,10 +22,8 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.Envelope;
-import com.esri.core.geometry.Geometry;
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Envelope;
+import org.locationtech.jts.geom.Geometry;
 
 public class STXMaxDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -33,15 +31,13 @@
     public static final IFunctionDescriptorFactory FACTORY = STXMaxDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        Geometry esriGeom = geometry.getEsriGeometry();
-        if (esriGeom != null) {
-            Envelope env = new Envelope();
-            esriGeom.queryEnvelope(env);
-            return env.getXMax();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        Envelope env = geometry.getEnvelopeInternal();
+        if (env != null) {
+            return env.getMaxX();
         } else {
-            throw new UnsupportedOperationException(
-                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+            throw new UnsupportedOperationException("The operation " + getIdentifier()
+                    + " is not supported for the type " + geometry.getGeometryType());
         }
 
     }
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMinDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMinDescriptor.java
index 66db0c6..ad47b30 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMinDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMinDescriptor.java
@@ -22,10 +22,8 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.Envelope;
-import com.esri.core.geometry.Geometry;
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Envelope;
+import org.locationtech.jts.geom.Geometry;
 
 public class STXMinDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -33,17 +31,14 @@
     public static final IFunctionDescriptorFactory FACTORY = STXMinDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        Geometry esriGeom = geometry.getEsriGeometry();
-        if (esriGeom != null) {
-            Envelope env = new Envelope();
-            esriGeom.queryEnvelope(env);
-            return env.getXMin();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        Envelope env = geometry.getEnvelopeInternal();
+        if (env != null) {
+            return env.getMinX();
         } else {
-            throw new UnsupportedOperationException(
-                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+            throw new UnsupportedOperationException("The operation " + getIdentifier()
+                    + " is not supported for the type " + geometry.getGeometryType());
         }
-
     }
 
     @Override
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYDescriptor.java
index 32baa0a..df5a57d 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYDescriptor.java
@@ -20,11 +20,11 @@
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
-import com.esri.core.geometry.ogc.OGCPoint;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.Point;
 
 public class STYDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -32,12 +32,12 @@
     public static final IFunctionDescriptorFactory FACTORY = STYDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        if (geometry instanceof OGCPoint) {
-            return ((OGCPoint) geometry).Y();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        if (StringUtils.equals(geometry.getGeometryType(), Geometry.TYPENAME_POINT)) {
+            return ((Point) geometry).getY();
         } else {
-            throw new UnsupportedOperationException(
-                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+            throw new UnsupportedOperationException("The operation " + getIdentifier()
+                    + " is not supported for the type " + geometry.getGeometryType());
         }
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMaxDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMaxDescriptor.java
index 7b82915..b5aee1f 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMaxDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMaxDescriptor.java
@@ -22,10 +22,8 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.Envelope;
-import com.esri.core.geometry.Geometry;
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Envelope;
+import org.locationtech.jts.geom.Geometry;
 
 public class STYMaxDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -33,15 +31,13 @@
     public static final IFunctionDescriptorFactory FACTORY = STYMaxDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        Geometry esriGeom = geometry.getEsriGeometry();
-        if (esriGeom != null) {
-            Envelope env = new Envelope();
-            esriGeom.queryEnvelope(env);
-            return env.getYMax();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        Envelope env = geometry.getEnvelopeInternal();
+        if (env != null) {
+            return env.getMaxY();
         } else {
-            throw new UnsupportedOperationException(
-                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+            throw new UnsupportedOperationException("The operation " + getIdentifier()
+                    + " is not supported for the type " + geometry.getGeometryType());
         }
 
     }
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMinDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMinDescriptor.java
index 1cc5e88..921761f 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMinDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMinDescriptor.java
@@ -22,10 +22,8 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.Envelope;
-import com.esri.core.geometry.Geometry;
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Envelope;
+import org.locationtech.jts.geom.Geometry;
 
 public class STYMinDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -33,15 +31,13 @@
     public static final IFunctionDescriptorFactory FACTORY = STYMinDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        Geometry esriGeom = geometry.getEsriGeometry();
-        if (esriGeom != null) {
-            Envelope env = new Envelope();
-            esriGeom.queryEnvelope(env);
-            return env.getYMin();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        Envelope env = geometry.getEnvelopeInternal();
+        if (env != null) {
+            return env.getMinY();
         } else {
-            throw new UnsupportedOperationException(
-                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+            throw new UnsupportedOperationException("The operation " + getIdentifier()
+                    + " is not supported for the type " + geometry.getGeometryType());
         }
 
     }
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZDescriptor.java
index 739775a..eae150c 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZDescriptor.java
@@ -20,24 +20,32 @@
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
-import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.Point;
 
 public class STZDescriptor extends AbstractSTSingleGeometryDescriptor {
 
     private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = STZDescriptor::new;
+    private static final Logger LOGGER = LogManager.getLogger();
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        if (geometry instanceof OGCPoint) {
-            return ((OGCPoint) geometry).Z();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        if (StringUtils.equals(geometry.getGeometryType(), Geometry.TYPENAME_POINT)) {
+            Point point = (Point) geometry;
+            double z = point.getCoordinate().getZ();
+            if (Double.isNaN(z)) {
+                LOGGER.debug("No Z-coordinate available for the given point.");
+            }
+            return z;
         } else {
-            throw new UnsupportedOperationException(
-                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+            throw new UnsupportedOperationException("The operation " + getIdentifier()
+                    + " is not supported for the type " + geometry.getGeometryType());
         }
     }
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMaxDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMaxDescriptor.java
index cb3f54e..fa80241 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMaxDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMaxDescriptor.java
@@ -22,8 +22,8 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.Geometry;
 
 public class STZMaxDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -31,8 +31,18 @@
     public static final IFunctionDescriptorFactory FACTORY = STZMaxDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        return geometry.MaxZ();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        double maxZ = Double.NEGATIVE_INFINITY;
+        boolean foundZ = false;
+        for (Coordinate coord : geometry.getCoordinates()) {
+            if (!Double.isNaN(coord.getZ())) {
+                foundZ = true;
+                if (coord.getZ() > maxZ) {
+                    maxZ = coord.getZ();
+                }
+            }
+        }
+        return foundZ ? maxZ : Double.NaN;
     }
 
     @Override
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMinDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMinDescriptor.java
index 3f6f6b3..ba637d4 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMinDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMinDescriptor.java
@@ -22,8 +22,8 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.Geometry;
 
 public class STZMinDescriptor extends AbstractSTSingleGeometryDescriptor {
 
@@ -31,8 +31,19 @@
     public static final IFunctionDescriptorFactory FACTORY = STZMinDescriptor::new;
 
     @Override
-    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
-        return geometry.MinZ();
+    protected Object evaluateOGCGeometry(Geometry geometry) throws HyracksDataException {
+        double minZ = Double.POSITIVE_INFINITY;
+        boolean foundZ = false;
+        for (Coordinate coord : geometry.getCoordinates()) {
+            if (!Double.isNaN(coord.getZ())) {
+                foundZ = true;
+                if (coord.getZ() < minZ) {
+                    minZ = coord.getZ();
+                }
+            }
+        }
+
+        return foundZ ? minZ : Double.NaN;
     }
 
     @Override
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CopyToStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CopyToStatement.java
index 599d528..5c89a9f 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CopyToStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CopyToStatement.java
@@ -32,6 +32,8 @@
 import org.apache.asterix.lang.common.base.IReturningStatement;
 import org.apache.asterix.lang.common.clause.OrderbyClause;
 import org.apache.asterix.lang.common.expression.LiteralExpr;
+import org.apache.asterix.lang.common.expression.RecordTypeDefinition;
+import org.apache.asterix.lang.common.expression.TypeExpression;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.literal.StringLiteral;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
@@ -52,13 +54,15 @@
     private List<Expression> partitionExpressions;
     private List<Expression> orderByList;
     private int varCounter;
+    private RecordTypeDefinition itemType;
+    private TypeExpression typeExpressionItemType;
 
     public CopyToStatement(Namespace namespace, String datasetName, Query query, VariableExpr sourceVariable,
             ExternalDetailsDecl externalDetailsDecl, int varCounter, List<Expression> keyExpressions,
             boolean autogenerated) {
         this(namespace, datasetName, query, sourceVariable, externalDetailsDecl, new ArrayList<>(), new ArrayList<>(),
                 new HashMap<>(), new ArrayList<>(), new ArrayList<>(), new ArrayList<>(), varCounter, keyExpressions,
-                autogenerated);
+                autogenerated, null, null);
     }
 
     public CopyToStatement(Namespace namespace, String datasetName, Query query, VariableExpr sourceVariable,
@@ -68,7 +72,18 @@
             List<OrderbyClause.NullOrderModifier> orderByNullModifierList, int varCounter) {
         this(namespace, datasetName, query, sourceVariable, externalDetailsDecl, pathExpressions, partitionExpressions,
                 partitionsVariables, orderbyList, orderByModifiers, orderByNullModifierList, varCounter,
-                new ArrayList<>(), false);
+                new ArrayList<>(), false, null, null);
+    }
+
+    public CopyToStatement(Namespace namespace, String datasetName, Query query, VariableExpr sourceVariable,
+            ExternalDetailsDecl externalDetailsDecl, List<Expression> pathExpressions,
+            List<Expression> partitionExpressions, Map<Integer, VariableExpr> partitionsVariables,
+            List<Expression> orderbyList, List<OrderbyClause.OrderModifier> orderByModifiers,
+            List<OrderbyClause.NullOrderModifier> orderByNullModifierList, int varCounter,
+            TypeExpression typeExpressionItemType, RecordTypeDefinition itemType) {
+        this(namespace, datasetName, query, sourceVariable, externalDetailsDecl, pathExpressions, partitionExpressions,
+                partitionsVariables, orderbyList, orderByModifiers, orderByNullModifierList, varCounter,
+                new ArrayList<>(), false, typeExpressionItemType, itemType);
     }
 
     private CopyToStatement(Namespace namespace, String datasetName, Query query, VariableExpr sourceVariable,
@@ -76,7 +91,8 @@
             List<Expression> partitionExpressions, Map<Integer, VariableExpr> partitionsVariables,
             List<Expression> orderbyList, List<OrderbyClause.OrderModifier> orderByModifiers,
             List<OrderbyClause.NullOrderModifier> orderByNullModifierList, int varCounter,
-            List<Expression> keyExpressions, boolean autogenerated) {
+            List<Expression> keyExpressions, boolean autogenerated, TypeExpression typeExpressionItemType,
+            RecordTypeDefinition itemType) {
         this.namespace = namespace;
         this.datasetName = datasetName;
         this.query = query;
@@ -91,6 +107,8 @@
         this.varCounter = varCounter;
         this.keyExpressions = keyExpressions != null ? keyExpressions : new ArrayList<>();
         this.autogenerated = autogenerated;
+        this.itemType = itemType;
+        this.typeExpressionItemType = typeExpressionItemType;
 
         if (pathExpressions.isEmpty()) {
             // Ensure path expressions to have at least an empty string
@@ -117,6 +135,10 @@
         this.namespace = namespace;
     }
 
+    public RecordTypeDefinition getType() {
+        return itemType;
+    }
+
     public Namespace getNamespace() {
         return namespace;
     }
@@ -192,6 +214,10 @@
         return !orderByList.isEmpty();
     }
 
+    public TypeExpression getTypeExpressionItemType() {
+        return typeExpressionItemType;
+    }
+
     @Override
     public int getVarCounter() {
         return varCounter;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ExternalDetailsDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ExternalDetailsDecl.java
index db599c4..e1c978a 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ExternalDetailsDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ExternalDetailsDecl.java
@@ -20,9 +20,12 @@
 
 import java.util.Map;
 
+import org.apache.asterix.om.types.ARecordType;
+
 public class ExternalDetailsDecl implements IDatasetDetailsDecl {
     private Map<String, String> properties;
     private String adapter;
+    private ARecordType itemType;
 
     public void setAdapter(String adapter) {
         this.adapter = adapter;
@@ -32,6 +35,14 @@
         this.properties = properties;
     }
 
+    public void setItemType(ARecordType itemType) {
+        this.itemType = itemType;
+    }
+
+    public ARecordType getItemType() {
+        return itemType;
+    }
+
     public String getAdapter() {
         return adapter;
     }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ExpressionUtils.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ExpressionUtils.java
index c9f8640..df7d6d8 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ExpressionUtils.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ExpressionUtils.java
@@ -263,12 +263,17 @@
                             Triple<DatasetFullyQualifiedName, Boolean, DatasetFullyQualifiedName> dsArgs =
                                     FunctionUtil.parseDatasetFunctionArguments(functionCall);
                             DatasetFullyQualifiedName datasetFullyQualifiedName = dsArgs.first;
-                            EntityDetails.EntityType entityType =
-                                    dsArgs.second ? EntityDetails.EntityType.VIEW : EntityDetails.EntityType.DATASET;
-                            metadataProvider
-                                    .addAccessedEntity(new EntityDetails(datasetFullyQualifiedName.getDatabaseName(),
-                                            datasetFullyQualifiedName.getDataverseName(),
-                                            datasetFullyQualifiedName.getDatasetName(), entityType));
+                            if (dsArgs.second) {
+                                metadataProvider.addAccessedEntity(
+                                        EntityDetails.newView(datasetFullyQualifiedName.getDatabaseName(),
+                                                datasetFullyQualifiedName.getDataverseName(),
+                                                datasetFullyQualifiedName.getDatasetName()));
+                            } else {
+                                metadataProvider.addAccessedEntity(
+                                        EntityDetails.newDataset(datasetFullyQualifiedName.getDatabaseName(),
+                                                datasetFullyQualifiedName.getDataverseName(),
+                                                datasetFullyQualifiedName.getDatasetName()));
+                            }
                             DatasetFullyQualifiedName synonymReference = dsArgs.third;
                             if (synonymReference != null) {
                                 // resolved via synonym -> store synonym name as a dependency
@@ -290,8 +295,8 @@
                     } else {
                         if (seenFunctions.add(signature)) {
                             String functionName = signature.getName() + "(" + signature.getArity() + ")";
-                            metadataProvider.addAccessedEntity(new EntityDetails(signature.getDatabaseName(),
-                                    signature.getDataverseName(), functionName, EntityDetails.EntityType.FUNCTION));
+                            metadataProvider.addAccessedEntity(EntityDetails.newFunction(signature.getDatabaseName(),
+                                    signature.getDataverseName(), functionName, signature.getArity()));
                             outFunctionDependencies.add(new DependencyFullyQualifiedName(signature.getDatabaseName(),
                                     signature.getDataverseName(), signature.getName(),
                                     Integer.toString(signature.getArity())));
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
index f371881..7f6e0ab 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
@@ -19,6 +19,12 @@
 
 package org.apache.asterix.lang.sqlpp.rewrites.visitor;
 
+import static org.apache.asterix.common.utils.ConstantUtil.LIKE_ESCAPE;
+import static org.apache.asterix.common.utils.ConstantUtil.PERCENT;
+import static org.apache.asterix.common.utils.ConstantUtil.UNDERSCORE;
+import static org.apache.asterix.lang.sqlpp.rewrites.visitor.OperatorExpressionVisitor.LikePattern.EQUAL;
+import static org.apache.asterix.lang.sqlpp.rewrites.visitor.OperatorExpressionVisitor.LikePattern.PREFIX;
+
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
@@ -34,10 +40,12 @@
 import org.apache.asterix.lang.common.expression.QuantifiedExpression.Quantifier;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.literal.FalseLiteral;
+import org.apache.asterix.lang.common.literal.StringLiteral;
 import org.apache.asterix.lang.common.literal.TrueLiteral;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.struct.OperatorType;
 import org.apache.asterix.lang.common.struct.QuantifiedPair;
+import org.apache.asterix.lang.common.util.ExpressionUtils;
 import org.apache.asterix.lang.sqlpp.expression.CaseExpression;
 import org.apache.asterix.lang.sqlpp.util.FunctionMapUtil;
 import org.apache.asterix.lang.sqlpp.util.SqlppRewriteUtil;
@@ -82,17 +90,27 @@
         return operatorExpr;
     }
 
-    private Expression processLikeOperator(OperatorExpr operatorExpr, OperatorType opType) {
-        CallExpr likeExpr =
-                new CallExpr(new FunctionSignature(BuiltinFunctions.STRING_LIKE), operatorExpr.getExprList());
-        likeExpr.addHints(operatorExpr.getHints());
-        likeExpr.setSourceLocation(operatorExpr.getSourceLocation());
+    private Expression processLikeOperator(OperatorExpr operatorExpr, OperatorType opType) throws CompilationException {
         switch (opType) {
             case LIKE:
-                return likeExpr;
+                Expression target = operatorExpr.getExprList().get(0);
+                Expression patternExpr = operatorExpr.getExprList().get(1);
+                String patternStr = ExpressionUtils.getStringLiteral(patternExpr);
+                if (patternStr != null) {
+                    StringBuilder likePatternStr = new StringBuilder();
+                    LikePattern likePattern = processPattern(patternStr, likePatternStr);
+                    if (likePattern == PREFIX) {
+                        return convertLikeToRange(operatorExpr, target, likePatternStr.toString());
+                    } else if (likePattern == EQUAL) {
+                        Expression processedExpr = new LiteralExpr(new StringLiteral(likePatternStr.toString()));
+                        return createOperatorExpression(OperatorType.EQ, target, processedExpr, operatorExpr.getHints(),
+                                operatorExpr.getSourceLocation());
+                    }
+                }
+                return createLikeExpression(operatorExpr);
             case NOT_LIKE:
                 CallExpr notLikeExpr = new CallExpr(new FunctionSignature(BuiltinFunctions.NOT),
-                        new ArrayList<>(Collections.singletonList(likeExpr)));
+                        new ArrayList<>(Collections.singletonList(createLikeExpression(operatorExpr))));
                 notLikeExpr.setSourceLocation(operatorExpr.getSourceLocation());
                 return notLikeExpr;
             default:
@@ -156,19 +174,8 @@
         Expression target = operatorExpr.getExprList().get(0);
         Expression left = operatorExpr.getExprList().get(1);
         Expression right = operatorExpr.getExprList().get(2);
+        Expression andExpr = createRangeExpression(target, OperatorType.GE, left, right, OperatorType.LE, operatorExpr);
 
-        // Creates the expression target >= left.
-        Expression leftComparison = createOperatorExpression(OperatorType.GE, target, left, operatorExpr.getHints(),
-                operatorExpr.getSourceLocation());
-        // Creates the expression target <= right.
-        Expression targetCopy = (Expression) SqlppRewriteUtil.deepCopy(target);
-
-        // remove any selectivity hints from operatorExpr; do not want to duplicate those hints
-        Expression rightComparison = createOperatorExpression(OperatorType.LE, targetCopy, right,
-                removeSelectivityHints(operatorExpr), operatorExpr.getSourceLocation());
-
-        Expression andExpr = createOperatorExpression(OperatorType.AND, leftComparison, rightComparison, null,
-                operatorExpr.getSourceLocation());
         switch (opType) {
             case BETWEEN:
                 return andExpr;
@@ -261,4 +268,67 @@
         }
         return comparison;
     }
+
+    private Expression createRangeExpression(Expression target, OperatorType leftOp, Expression left, Expression right,
+            OperatorType rightOp, OperatorExpr operatorExpr) throws CompilationException {
+        Expression leftComparison = createOperatorExpression(leftOp, target, left, operatorExpr.getHints(),
+                operatorExpr.getSourceLocation());
+        Expression targetCopy = (Expression) SqlppRewriteUtil.deepCopy(target);
+
+        // remove any selectivity hints from operatorExpr; do not want to duplicate those hints
+        Expression rightComparison = createOperatorExpression(rightOp, targetCopy, right,
+                removeSelectivityHints(operatorExpr), operatorExpr.getSourceLocation());
+
+        Expression andExpr = createOperatorExpression(OperatorType.AND, leftComparison, rightComparison, null,
+                operatorExpr.getSourceLocation());
+        return andExpr;
+    }
+
+    private Expression convertLikeToRange(OperatorExpr operatorExpr, Expression target, String prefix)
+            throws CompilationException {
+        int lastCodePoint = prefix.codePointAt(prefix.length() - 1);
+        String incrementedLastChar = new String(Character.toChars(lastCodePoint + 1));
+        String incrementedStr = prefix.substring(0, prefix.length() - 1) + incrementedLastChar;
+        Expression left = new LiteralExpr(new StringLiteral(prefix));
+        Expression right = new LiteralExpr(new StringLiteral(incrementedStr));
+        return createRangeExpression(target, OperatorType.GE, left, right, OperatorType.LT, operatorExpr);
+    }
+
+    private static CallExpr createLikeExpression(OperatorExpr operatorExpr) {
+        CallExpr likeExpr =
+                new CallExpr(new FunctionSignature(BuiltinFunctions.STRING_LIKE), operatorExpr.getExprList());
+        likeExpr.addHints(operatorExpr.getHints());
+        likeExpr.setSourceLocation(operatorExpr.getSourceLocation());
+        return likeExpr;
+    }
+
+    private static LikePattern processPattern(String pattern, StringBuilder likePatternStr) {
+        // note: similar logic is applied in StringLikeDescriptor
+        LikePattern likePattern = EQUAL;
+        for (int i = 0, length = pattern.length(); i < length; i++) {
+            char c = pattern.charAt(i);
+            if (c == LIKE_ESCAPE) {
+                char nextChar;
+                // escape character can't be last, and only %, _ and the escape char are allowed after it
+                if (i >= length - 1 || ((nextChar = pattern.charAt(i + 1)) != PERCENT && nextChar != UNDERSCORE
+                        && nextChar != LIKE_ESCAPE)) {
+                    return null;
+                }
+                likePatternStr.append(nextChar);
+                ++i;
+            } else if (c == PERCENT && i == pattern.length() - 1) {
+                likePattern = PREFIX;
+            } else if (c == UNDERSCORE || c == PERCENT) {
+                return null;
+            } else {
+                likePatternStr.append(c);
+            }
+        }
+        return likePattern;
+    }
+
+    enum LikePattern {
+        PREFIX,
+        EQUAL
+    }
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppLoadAccessedDataset.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppLoadAccessedDataset.java
index 2f3323f..1b1cf83 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppLoadAccessedDataset.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppLoadAccessedDataset.java
@@ -84,8 +84,13 @@
                 }
             }
 
-            context.getMetadataProvider()
-                    .addAccessedEntity(new EntityDetails(databaseName, dataverseName, datasetName, entityType));
+            if (entityType == EntityDetails.EntityType.VIEW) {
+                context.getMetadataProvider()
+                        .addAccessedEntity(EntityDetails.newView(databaseName, dataverseName, datasetName));
+            } else {
+                context.getMetadataProvider()
+                        .addAccessedEntity(EntityDetails.newDataset(databaseName, dataverseName, datasetName));
+            }
 
         } else {
             FunctionSignature signature = expression.getFunctionSignature();
@@ -93,9 +98,9 @@
             if (declaredFunctions.containsKey(signature)) {
                 return;
             }
-            String functionName = signature.getName() + "(" + signature.getArity() + ")";
-            context.getMetadataProvider().addAccessedEntity(new EntityDetails(signature.getDatabaseName(),
-                    signature.getDataverseName(), functionName, EntityDetails.EntityType.FUNCTION));
+            String functionName = EntityDetails.getFunctionNameWithArity(signature.getName(), signature.getArity());
+            context.getMetadataProvider().addAccessedEntity(EntityDetails.newFunction(signature.getDatabaseName(),
+                    signature.getDataverseName(), functionName, signature.getArity()));
         }
     }
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index da5f0ab..6b3aa54 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -2936,6 +2936,9 @@
   Namespace namespace = nameComponents == null ? null : nameComponents.first;
   String datasetName = nameComponents == null ? null : nameComponents.second.getValue();
   List<Expression> pathExprs;
+  RecordTypeDefinition recordTypeDefinition = null;
+  TypeExpression typeExpr = null;
+  Boolean isRecordTypeDefinition = false;
 
   List<Expression> partitionExprs = new ArrayList<Expression>();
   Map<Integer, VariableExpr> partitionVarExprs = new HashMap<Integer, VariableExpr>();
@@ -2947,6 +2950,21 @@
   <TO> adapterName = AdapterName()
   <PATH> <LEFTPAREN> pathExprs = ExpressionList() <RIGHTPAREN>
   (CopyToOverClause(partitionExprs, partitionVarExprs, orderbyList, orderbyModifierList, orderbyNullModifierList))?
+  (<TYPE> <LEFTPAREN>
+    {
+      recordTypeDefinition = RecordTypeDef();
+      isRecordTypeDefinition = true;
+    }
+    <RIGHTPAREN>) ?
+  (<AS>
+    {
+      if (isRecordTypeDefinition == false) {
+        typeExpr = DatasetRecordTypeSpecification(false, null);
+      } else {
+        throw new SqlppParseException(getSourceLocation(token), "Syntax error: Both 'TYPE()' and 'AS()' are provided. Please use either 'TYPE()' or 'AS()'.");
+      }
+    }
+  )?
   <WITH> withRecord = RecordConstructor()
     {
        ExternalDetailsDecl edd = new ExternalDetailsDecl();
@@ -2961,8 +2979,7 @@
           usedAlias = new VariableExpr(SqlppVariableUtil.toInternalVariableIdentifier(datasetName));
        }
 
-       CopyToStatement stmt = new CopyToStatement(namespace, datasetName, query, usedAlias, edd, pathExprs,
-            partitionExprs, partitionVarExprs, orderbyList, orderbyModifierList, orderbyNullModifierList, getVarCounter());
+       CopyToStatement stmt = new CopyToStatement(namespace, datasetName, query, usedAlias, edd, pathExprs, partitionExprs, partitionVarExprs, orderbyList, orderbyModifierList, orderbyNullModifierList, getVarCounter(), typeExpr, recordTypeDefinition);
        return addSourceLocation(stmt, startToken);
     }
 }
@@ -4167,6 +4184,7 @@
   OperatorExpr op = null;
   Expression operand = null;
   IExpressionAnnotation annotation = null;
+  List<IExpressionAnnotation> annotationList = null;
 }
 {
     operand = ConcatExpr()
@@ -4174,9 +4192,17 @@
         LOOKAHEAD(2)
         (<NOT> { not = true; })? <LIKE>
         {
-          Token hintToken = fetchHint(token, SqlppHint.SINGLE_DATASET_PREDICATE_SELECTIVITY_HINT);
-          if (hintToken != null) {
-               annotation = parseExpressionAnnotation(hintToken);
+          List<Token> hintTokens = fetchHints(token, SqlppHint.SINGLE_DATASET_PREDICATE_SELECTIVITY_HINT,
+            SqlppHint.SKIP_SECONDARY_INDEX_SEARCH_HINT, SqlppHint.USE_SECONDARY_INDEX_SEARCH_HINT);
+          if (hintTokens != null && !hintTokens.isEmpty()) {
+            annotationList = new ArrayList<IExpressionAnnotation>();
+          }
+          for (Token hintToken : hintTokens) {
+            annotation = parseExpressionAnnotation(hintToken);
+            if (annotation != null) {
+              // annotation may be null if hints are malformed
+              annotationList.add(annotation);
+            }
           }
           op = new OperatorExpr();
           op.addOperand(operand);
@@ -4192,8 +4218,8 @@
           } catch (CompilationException e){
             throw new SqlppParseException(getSourceLocation(token), e.getMessage());
           }
-          if (annotation != null) {
-             op.addHint(annotation);
+          if (annotationList != null && !annotationList.isEmpty()) {
+            op.addHints(annotationList);
           }
         }
 
@@ -4374,9 +4400,11 @@
 {
   expr = PrimaryExpr()
   (
-    accessor = FieldAccessor(accessor != null ? accessor : expr)
-    |
-    accessor = IndexAccessor(accessor != null ? accessor : expr)
+    LOOKAHEAD(2)(
+      accessor = FieldAccessor(accessor != null ? accessor : expr)
+      |
+      accessor = IndexAccessor(accessor != null ? accessor : expr)
+    )
   )*
   {
     return accessor == null ? expr : accessor;
@@ -4407,7 +4435,7 @@
   (
     <MUL> { star = true; }
     |
-    ( expr1 = Expression() ( <COLON> { slice = true; } ( expr2 = Expression() )? )? )
+    ( (expr1 = Expression()) ? ( <COLON> { slice = true; } ( expr2 = Expression() )? )? )
   )
   <RIGHTBRACKET>
   {
@@ -4418,10 +4446,14 @@
       ensureIntegerLiteral( (LiteralExpr) expr2, "Index");
     }
     AbstractAccessor resultAccessor;
-    if (slice) {
+
+    if (star) {
+          resultAccessor = new IndexAccessor(inputExpr, IndexAccessor.IndexKind.STAR, null);
+    } else if (slice || expr1 == null) {
+      if (expr1 == null) {
+        expr1 = new LiteralExpr(new LongIntegerLiteral(0L));
+      }
       resultAccessor = new ListSliceExpression(inputExpr, expr1, expr2);
-    } else if (star) {
-      resultAccessor = new IndexAccessor(inputExpr, IndexAccessor.IndexKind.STAR, null);
     } else {
       resultAccessor = new IndexAccessor(inputExpr, IndexAccessor.IndexKind.ELEMENT, expr1);
     }
@@ -5314,8 +5346,8 @@
 {
   (
     <MUL> { kind = Projection.Kind.STAR; startSrcLoc = getSourceLocation(token); }
-    | LOOKAHEAD(3) expr = VariableRef() <DOT> <MUL> { kind = Projection.Kind.VAR_STAR; }
-    | expr = Expression()
+        | LOOKAHEAD(ValueExpr() <DOT> <MUL>) expr = ValueExpr() <DOT> <MUL> { kind = Projection.Kind.VAR_STAR; }
+        | expr = Expression()
       ( // EXCLUDE is a soft-keyword-- we want to avoid mistaking EXCLUDE as an identifier here.
         LOOKAHEAD({ getToken(1).kind == AS || getToken(1).kind == QUOTED_STRING
                     || (getToken(1).kind == IDENTIFIER && !laIdentifier(1, EXCLUDE)) })
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
index 0eb1111..8399f96 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
@@ -451,7 +451,7 @@
                 InternalDatasetDetails id = (InternalDatasetDetails) dataset.getDatasetDetails();
                 Index primaryIndex = Index.createPrimaryIndex(dataset.getDatabaseName(), dataset.getDataverseName(),
                         dataset.getDatasetName(), id.getPrimaryKey(), id.getKeySourceIndicator(),
-                        id.getPrimaryKeyType(), dataset.getPendingOp());
+                        id.getPrimaryKeyType(), dataset.getPendingOp(), dataset.getCreator());
 
                 addIndex(txnId, primaryIndex);
             }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
index aeb6d13..70bf83a 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
@@ -165,26 +165,26 @@
 
     public void dropDataset(String database, DataverseName dataverseName, String datasetName) {
         Dataset dataset = new Dataset(database, dataverseName, datasetName, null, null, null, null, null, null, null,
-                null, null, -1, MetadataUtil.PENDING_NO_OP);
+                null, null, -1, MetadataUtil.PENDING_NO_OP, null);
         droppedCache.addDatasetIfNotExists(dataset);
         logAndApply(new MetadataLogicalOperation(dataset, false));
     }
 
     public void dropIndex(String database, DataverseName dataverseName, String datasetName, String indexName) {
         Index index = new Index(database, dataverseName, datasetName, indexName, null, null, false, false,
-                MetadataUtil.PENDING_NO_OP);
+                MetadataUtil.PENDING_NO_OP, null);
         droppedCache.addIndexIfNotExists(index);
         logAndApply(new MetadataLogicalOperation(index, false));
     }
 
     public void dropDatabase(String databaseName) {
-        Database database = new Database(databaseName, false, MetadataUtil.PENDING_NO_OP);
+        Database database = new Database(databaseName, false, MetadataUtil.PENDING_NO_OP, null);
         droppedCache.addDatabaseIfNotExists(database);
         logAndApply(new MetadataLogicalOperation(database, false));
     }
 
     public void dropDataverse(String database, DataverseName dataverseName) {
-        Dataverse dataverse = new Dataverse(database, dataverseName, null, MetadataUtil.PENDING_NO_OP);
+        Dataverse dataverse = new Dataverse(database, dataverseName, null, MetadataUtil.PENDING_NO_OP, null);
         droppedCache.addDataverseIfNotExists(dataverse);
         logAndApply(new MetadataLogicalOperation(dataverse, false));
     }
@@ -203,7 +203,7 @@
 
     public void dropFunction(FunctionSignature signature) {
         Function function = new Function(signature, null, null, null, null, null, null, null, null, null, null, false,
-                false, null, null);
+                false, null, null, null);
         droppedCache.addFunctionIfNotExists(function);
         logAndApply(new MetadataLogicalOperation(function, false));
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
index 71458f9..901ffdf 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -71,6 +71,7 @@
 import org.apache.asterix.metadata.entities.Node;
 import org.apache.asterix.metadata.entities.NodeGroup;
 import org.apache.asterix.metadata.feeds.BuiltinFeedPolicies;
+import org.apache.asterix.metadata.utils.Creator;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.BuiltinTypeMap;
 import org.apache.asterix.om.types.IAType;
@@ -221,13 +222,12 @@
             IDatasetDetails id = new InternalDatasetDetails(FileStructure.BTREE, PartitioningStrategy.HASH,
                     indexes[i].getPartitioningExpr(), indexes[i].getPartitioningExpr(), null,
                     indexes[i].getPartitioningExprType(), false, null, null);
-            MetadataManager.INSTANCE.addDataset(mdTxnCtx,
-                    new Dataset(indexes[i].getDatabaseName(), indexes[i].getDataverseName(),
-                            indexes[i].getIndexedDatasetName(), indexes[i].getDatabaseName(),
-                            indexes[i].getDataverseName(), indexes[i].getPayloadRecordType().getTypeName(),
-                            indexes[i].getNodeGroupName(), StorageConstants.DEFAULT_COMPACTION_POLICY_NAME,
-                            StorageConstants.DEFAULT_COMPACTION_POLICY_PROPERTIES, id, new HashMap<>(),
-                            DatasetType.INTERNAL, indexes[i].getDatasetId().getId(), MetadataUtil.PENDING_NO_OP));
+            MetadataManager.INSTANCE.addDataset(mdTxnCtx, new Dataset(indexes[i].getDatabaseName(),
+                    indexes[i].getDataverseName(), indexes[i].getIndexedDatasetName(), indexes[i].getDatabaseName(),
+                    indexes[i].getDataverseName(), indexes[i].getPayloadRecordType().getTypeName(),
+                    indexes[i].getNodeGroupName(), StorageConstants.DEFAULT_COMPACTION_POLICY_NAME,
+                    StorageConstants.DEFAULT_COMPACTION_POLICY_PROPERTIES, id, new HashMap<>(), DatasetType.INTERNAL,
+                    indexes[i].getDatasetId().getId(), MetadataUtil.PENDING_NO_OP, Creator.DEFAULT_CREATOR));
         }
         if (LOGGER.isInfoEnabled()) {
             LOGGER.info("Finished inserting initial datasets.");
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBuiltinEntities.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBuiltinEntities.java
index 7b33682..f11ab01 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBuiltinEntities.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBuiltinEntities.java
@@ -24,6 +24,7 @@
 import org.apache.asterix.metadata.entities.Database;
 import org.apache.asterix.metadata.entities.Datatype;
 import org.apache.asterix.metadata.entities.Dataverse;
+import org.apache.asterix.metadata.utils.Creator;
 import org.apache.asterix.om.utils.RecordUtil;
 import org.apache.asterix.runtime.formats.NonTaggedDataFormat;
 
@@ -31,18 +32,19 @@
 
     //--------------------------------------- Databases ----------------------------------------//
     public static final Database SYSTEM_DATABASE =
-            new Database(MetadataConstants.SYSTEM_DATABASE, true, MetadataUtil.PENDING_NO_OP);
+            new Database(MetadataConstants.SYSTEM_DATABASE, true, MetadataUtil.PENDING_NO_OP, Creator.DEFAULT_CREATOR);
 
-    public static final Database DEFAULT_DATABASE =
-            new Database(MetadataConstants.DEFAULT_DATABASE, false, MetadataUtil.PENDING_NO_OP);
+    public static final Database DEFAULT_DATABASE = new Database(MetadataConstants.DEFAULT_DATABASE, false,
+            MetadataUtil.PENDING_NO_OP, Creator.DEFAULT_CREATOR);
 
     //--------------------------------------- Dataverses ----------------------------------------//
     public static final Dataverse METADATA_DATAVERSE =
             new Dataverse(MetadataConstants.SYSTEM_DATABASE, MetadataConstants.METADATA_DATAVERSE_NAME,
-                    NonTaggedDataFormat.NON_TAGGED_DATA_FORMAT, MetadataUtil.PENDING_NO_OP);
+                    NonTaggedDataFormat.NON_TAGGED_DATA_FORMAT, MetadataUtil.PENDING_NO_OP, Creator.DEFAULT_CREATOR);
 
-    public static final Dataverse DEFAULT_DATAVERSE = new Dataverse(MetadataConstants.DEFAULT_DATABASE,
-            MetadataConstants.DEFAULT_DATAVERSE_NAME, NonTaggedDataFormat.class.getName(), MetadataUtil.PENDING_NO_OP);
+    public static final Dataverse DEFAULT_DATAVERSE =
+            new Dataverse(MetadataConstants.DEFAULT_DATABASE, MetadataConstants.DEFAULT_DATAVERSE_NAME,
+                    NonTaggedDataFormat.class.getName(), MetadataUtil.PENDING_NO_OP, Creator.DEFAULT_CREATOR);
 
     //--------------------------------------- Datatypes -----------------------------------------//
     public static final Datatype ANY_OBJECT_DATATYPE =
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
index c35c03a..338e00d 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
@@ -130,6 +130,11 @@
     public static final String FIELD_NAME_FULL_TEXT_FILTER_PIPELINE = "FullTextFilterPipeline";
     public static final String FIELD_NAME_FULL_TEXT_STOPWORD_LIST = "StopwordList";
 
+    //open field
+    public static final String FIELD_NAME_CREATOR_NAME = "Name";
+    public static final String FIELD_NAME_CREATOR_UUID = "Uuid";
+    public static final String CREATOR_ARECORD_FIELD_NAME = "Creator";
+
     //---------------------------------- Record Types Creation ----------------------------------//
     //--------------------------------------- Properties ----------------------------------------//
     public static final int PROPERTIES_NAME_FIELD_INDEX = 0;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/IExternalWriteDataSink.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/IExternalWriteDataSink.java
new file mode 100644
index 0000000..64f8d6d
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/IExternalWriteDataSink.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.metadata.declared;
+
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IWriteDataSink;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public interface IExternalWriteDataSink extends IWriteDataSink {
+    ARecordType getItemType();
+
+    SourceLocation getSourceLoc();
+}
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 cc456e5..ef056c9 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
@@ -100,7 +100,6 @@
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.asterix.runtime.base.AsterixTupleFilterFactory;
 import org.apache.asterix.runtime.formats.FormatUtils;
@@ -110,9 +109,6 @@
 import org.apache.asterix.runtime.operators.LSMSecondaryInsertDeleteWithNestedPlanOperatorDescriptor;
 import org.apache.asterix.runtime.operators.LSMSecondaryUpsertOperatorDescriptor;
 import org.apache.asterix.runtime.operators.LSMSecondaryUpsertWithNestedPlanOperatorDescriptor;
-import org.apache.asterix.runtime.writer.ExternalFileWriterFactory;
-import org.apache.asterix.runtime.writer.IExternalFileWriterFactory;
-import org.apache.asterix.runtime.writer.IExternalPrinterFactory;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -143,7 +139,6 @@
 import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.algebricks.runtime.operators.writer.SinkExternalWriterRuntimeFactory;
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
@@ -794,17 +789,9 @@
             IScalarEvaluatorFactory dynamicPathEvalFactory, ILogicalExpression staticPathExpr,
             SourceLocation pathSourceLocation, IWriteDataSink sink, RecordDescriptor inputDesc, Object sourceType)
             throws AlgebricksException {
-        String staticPath = staticPathExpr != null ? ConstantExpressionUtil.getStringConstant(staticPathExpr) : null;
-        IExternalFileWriterFactory fileWriterFactory =
-                ExternalWriterProvider.createWriterFactory(appCtx, sink, staticPath, pathSourceLocation);
-        fileWriterFactory.validate();
-        String fileExtension = ExternalWriterProvider.getFileExtension(sink);
-        int maxResult = ExternalWriterProvider.getMaxResult(sink);
-        IExternalPrinterFactory printerFactory = ExternalWriterProvider.createPrinter(appCtx, sink, sourceType);
-        ExternalFileWriterFactory writerFactory = new ExternalFileWriterFactory(fileWriterFactory, printerFactory,
-                fileExtension, maxResult, dynamicPathEvalFactory, staticPath, pathSourceLocation);
-        SinkExternalWriterRuntimeFactory runtime = new SinkExternalWriterRuntimeFactory(sourceColumn, partitionColumns,
-                partitionComparatorFactories, inputDesc, writerFactory);
+        IPushRuntimeFactory runtime = ExternalWriterProvider.getWriteFileRuntime(appCtx, sink, sourceType,
+                staticPathExpr, pathSourceLocation, dynamicPathEvalFactory, inputDesc, sourceColumn, partitionColumns,
+                partitionComparatorFactories);
         return new Pair<>(runtime, null);
     }
 
@@ -1007,6 +994,7 @@
             Map<String, String> configuration, ARecordType itemType, IWarningCollector warningCollector,
             IExternalFilterEvaluatorFactory filterEvaluatorFactory) throws AlgebricksException {
         try {
+            configuration.put(ExternalDataConstants.KEY_DATASET, dataset.getDatasetName());
             configuration.put(ExternalDataConstants.KEY_DATASET_DATABASE, dataset.getDatabaseName());
             configuration.put(ExternalDataConstants.KEY_DATASET_DATAVERSE,
                     dataset.getDataverseName().getCanonicalForm());
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/WriteDataSink.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/WriteDataSink.java
index 753ac54..d1667bf 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/WriteDataSink.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/WriteDataSink.java
@@ -21,20 +21,39 @@
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.asterix.om.types.ARecordType;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IWriteDataSink;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
-public class WriteDataSink implements IWriteDataSink {
+public class WriteDataSink implements IExternalWriteDataSink {
     private final String adapterName;
     private final Map<String, String> configuration;
+    private ARecordType itemType;
+    private SourceLocation sourceLoc;
 
-    public WriteDataSink(String adapterName, Map<String, String> configuration) {
+    public WriteDataSink(String adapterName, Map<String, String> configuration, ARecordType itemType,
+            SourceLocation sourceLoc) {
         this.adapterName = adapterName;
         this.configuration = configuration;
+        this.itemType = itemType;
+        this.sourceLoc = sourceLoc;
     }
 
     private WriteDataSink(WriteDataSink writeDataSink) {
         this.adapterName = writeDataSink.getAdapterName();
         this.configuration = new HashMap<>(writeDataSink.configuration);
+        this.itemType = writeDataSink.itemType;
+        this.sourceLoc = writeDataSink.sourceLoc;
+    }
+
+    @Override
+    public ARecordType getItemType() {
+        return itemType;
+    }
+
+    @Override
+    public SourceLocation getSourceLoc() {
+        return sourceLoc;
     }
 
     @Override
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Database.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Database.java
index 8be93e3..6336596 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Database.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Database.java
@@ -23,6 +23,7 @@
 
 import org.apache.asterix.metadata.MetadataCache;
 import org.apache.asterix.metadata.api.IMetadataEntity;
+import org.apache.asterix.metadata.utils.Creator;
 
 /**
  * Metadata describing a database.
@@ -34,11 +35,13 @@
     private final String databaseName;
     private final boolean isSystemDatabase;
     private final int pendingOp;
+    private final Creator creator;
 
-    public Database(String databaseName, boolean isSystemDatabase, int pendingOp) {
+    public Database(String databaseName, boolean isSystemDatabase, int pendingOp, Creator creator) {
         this.databaseName = databaseName;
         this.isSystemDatabase = isSystemDatabase;
         this.pendingOp = pendingOp;
+        this.creator = creator;
     }
 
     public String getDatabaseName() {
@@ -53,6 +56,10 @@
         return pendingOp;
     }
 
+    public Creator getCreator() {
+        return creator;
+    }
+
     @Override
     public Database addToCache(MetadataCache cache) {
         return cache.addDatabaseIfNotExists(this);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
index 132efbc..cae1385 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
@@ -63,6 +63,7 @@
 import org.apache.asterix.metadata.declared.ArrayBTreeResourceFactoryProvider;
 import org.apache.asterix.metadata.declared.BTreeResourceFactoryProvider;
 import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.utils.Creator;
 import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.metadata.utils.IndexUtil;
 import org.apache.asterix.metadata.utils.InvertedIndexResourceFactoryProvider;
@@ -166,15 +167,16 @@
     private final String compressionScheme;
     private final DatasetFullyQualifiedName datasetFullyQualifiedName;
     private final DatasetFormatInfo datasetFormatInfo;
+    private final Creator creator;
 
     public Dataset(String databaseName, DataverseName dataverseName, String datasetName, String recordTypeDatabaseName,
             DataverseName recordTypeDataverseName, String recordTypeName, String nodeGroupName, String compactionPolicy,
             Map<String, String> compactionPolicyProperties, IDatasetDetails datasetDetails, Map<String, String> hints,
-            DatasetType datasetType, int datasetId, int pendingOp) {
+            DatasetType datasetType, int datasetId, int pendingOp, Creator creator) {
         this(databaseName, dataverseName, datasetName, recordTypeDatabaseName, recordTypeDataverseName, recordTypeName,
                 null, /*metaTypeDataverseName*/null, /*metaTypeName*/null, nodeGroupName, compactionPolicy,
                 compactionPolicyProperties, datasetDetails, hints, datasetType, datasetId, pendingOp,
-                CompressionManager.NONE, DatasetFormatInfo.SYSTEM_DEFAULT);
+                CompressionManager.NONE, DatasetFormatInfo.SYSTEM_DEFAULT, creator);
     }
 
     public Dataset(String databaseName, DataverseName dataverseName, String datasetName, String itemTypeDatabaseName,
@@ -182,11 +184,11 @@
             DataverseName metaItemTypeDataverseName, String metaItemTypeName, String nodeGroupName,
             String compactionPolicy, Map<String, String> compactionPolicyProperties, IDatasetDetails datasetDetails,
             Map<String, String> hints, DatasetType datasetType, int datasetId, int pendingOp, String compressionScheme,
-            DatasetFormatInfo datasetFormatInfo) {
+            DatasetFormatInfo datasetFormatInfo, Creator creator) {
         this(databaseName, dataverseName, datasetName, itemTypeDatabaseName, itemTypeDataverseName, itemTypeName,
                 metaItemTypeDatabaseName, metaItemTypeDataverseName, metaItemTypeName, nodeGroupName, compactionPolicy,
                 compactionPolicyProperties, datasetDetails, hints, datasetType, datasetId, pendingOp, 0L,
-                compressionScheme, datasetFormatInfo);
+                compressionScheme, datasetFormatInfo, creator);
     }
 
     public Dataset(Dataset dataset) {
@@ -195,7 +197,7 @@
                 dataset.metaTypeDataverseName, dataset.metaTypeName, dataset.nodeGroupName,
                 dataset.compactionPolicyFactory, dataset.compactionPolicyProperties, dataset.datasetDetails,
                 dataset.hints, dataset.datasetType, dataset.datasetId, dataset.pendingOp, dataset.rebalanceCount,
-                dataset.compressionScheme, dataset.datasetFormatInfo);
+                dataset.compressionScheme, dataset.datasetFormatInfo, dataset.creator);
     }
 
     public Dataset(String databaseName, DataverseName dataverseName, String datasetName, String itemTypeDatabaseName,
@@ -203,7 +205,7 @@
             DataverseName metaItemTypeDataverseName, String metaItemTypeName, String nodeGroupName,
             String compactionPolicy, Map<String, String> compactionPolicyProperties, IDatasetDetails datasetDetails,
             Map<String, String> hints, DatasetType datasetType, int datasetId, int pendingOp, long rebalanceCount,
-            String compressionScheme, DatasetFormatInfo datasetFormatInfo) {
+            String compressionScheme, DatasetFormatInfo datasetFormatInfo, Creator creator) {
         this.databaseName = Objects.requireNonNull(databaseName);
         this.dataverseName = dataverseName;
         this.datasetName = datasetName;
@@ -225,6 +227,7 @@
         this.compressionScheme = compressionScheme;
         this.datasetFullyQualifiedName = new DatasetFullyQualifiedName(databaseName, dataverseName, datasetName);
         this.datasetFormatInfo = datasetFormatInfo;
+        this.creator = creator;
     }
 
     public String getDatabaseName() {
@@ -301,6 +304,10 @@
         return rebalanceCount;
     }
 
+    public Creator getCreator() {
+        return creator;
+    }
+
     public boolean hasMetaPart() {
         return metaTypeDataverseName != null && metaTypeName != null;
     }
@@ -395,7 +402,7 @@
                             getMetaItemTypeDataverseName(), getMetaItemTypeName(), getNodeGroupName(),
                             getCompactionPolicy(), getCompactionPolicyProperties(), getDatasetDetails(), getHints(),
                             getDatasetType(), getDatasetId(), MetadataUtil.PENDING_DROP_OP, getCompressionScheme(),
-                            getDatasetFormatInfo()));
+                            getDatasetFormatInfo(), creator));
 
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx.getValue());
             bActiveTxn.setValue(false);
@@ -856,7 +863,7 @@
                 this.metaTypeDataverseName, this.metaTypeName, targetNodeGroupName, this.compactionPolicyFactory,
                 this.compactionPolicyProperties, this.datasetDetails, this.hints, this.datasetType,
                 DatasetIdFactory.generateAlternatingDatasetId(this.datasetId), this.pendingOp, this.rebalanceCount + 1,
-                this.compressionScheme, this.datasetFormatInfo);
+                this.compressionScheme, this.datasetFormatInfo, creator);
     }
 
     // Gets an array of partition numbers for this dataset.
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataverse.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataverse.java
index bba1763..5b52be4 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataverse.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataverse.java
@@ -24,6 +24,7 @@
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.metadata.MetadataCache;
 import org.apache.asterix.metadata.api.IMetadataEntity;
+import org.apache.asterix.metadata.utils.Creator;
 
 /**
  * Metadata describing a dataverse.
@@ -35,12 +36,14 @@
     private final DataverseName dataverseName;
     private final String dataFormat;
     private final int pendingOp;
+    private final Creator creator;
 
-    public Dataverse(String databaseName, DataverseName dataverseName, String format, int pendingOp) {
+    public Dataverse(String databaseName, DataverseName dataverseName, String format, int pendingOp, Creator creator) {
         this.databaseName = Objects.requireNonNull(databaseName);
         this.dataverseName = dataverseName;
         this.dataFormat = format;
         this.pendingOp = pendingOp;
+        this.creator = creator;
     }
 
     public String getDatabaseName() {
@@ -59,6 +62,10 @@
         return pendingOp;
     }
 
+    public Creator getCreator() {
+        return creator;
+    }
+
     @Override
     public Dataverse addToCache(MetadataCache cache) {
         return cache.addDataverseIfNotExists(this);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/EntityDetails.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/EntityDetails.java
index a341a39..c7e771b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/EntityDetails.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/EntityDetails.java
@@ -20,26 +20,78 @@
 
 import org.apache.asterix.common.metadata.DataverseName;
 
+/**
+ * This class provides static factory methods for creating entity details.
+ */
+
 public class EntityDetails {
 
     public enum EntityType {
         DATASET,
         VIEW,
-        FUNCTION
+        FUNCTION,
+        DATABASE,
+        DATAVERSE,
+        SYNONYM,
+        INDEX
     }
 
     private final String databaseName;
     private final DataverseName dataverseName;
     private final String entityName;
     private final EntityType entityType;
+    // The number of arguments that the function accepts. Relevant only for function entity.
+    private int functionArity;
 
-    public EntityDetails(String databaseName, DataverseName dataverseName, String entityName, EntityType entityType) {
+    private EntityDetails(String databaseName, DataverseName dataverseName, String entityName, EntityType entityType) {
         this.databaseName = databaseName;
         this.dataverseName = dataverseName;
         this.entityName = entityName;
         this.entityType = entityType;
     }
 
+    private EntityDetails(String databaseName, DataverseName dataverseName, String entityName, EntityType entityType,
+            int functionArity) {
+        this.databaseName = databaseName;
+        this.dataverseName = dataverseName;
+        this.entityName = entityName;
+        this.entityType = entityType;
+        this.functionArity = functionArity;
+    }
+
+    public static EntityDetails newDatabase(String databaseName) {
+        return new EntityDetails(databaseName, null, null, EntityType.DATABASE);
+    }
+
+    public static EntityDetails newDataverse(String databaseName, DataverseName dataverseName) {
+        return new EntityDetails(databaseName, dataverseName, null, EntityType.DATAVERSE);
+    }
+
+    public static EntityDetails newDataset(String databaseName, DataverseName dataverseName, String datasetName) {
+        return new EntityDetails(databaseName, dataverseName, datasetName, EntityType.DATASET);
+    }
+
+    public static EntityDetails newView(String databaseName, DataverseName dataverseName, String viewName) {
+        return new EntityDetails(databaseName, dataverseName, viewName, EntityType.VIEW);
+    }
+
+    public static EntityDetails newFunction(String databaseName, DataverseName dataverseName, String functionName,
+            int functionArity) {
+        return new EntityDetails(databaseName, dataverseName, functionName, EntityType.FUNCTION, functionArity);
+    }
+
+    public static EntityDetails newSynonym(String databaseName, DataverseName dataverseName, String synonymName) {
+        return new EntityDetails(databaseName, dataverseName, synonymName, EntityType.SYNONYM);
+    }
+
+    public static EntityDetails newIndex(String databaseName, DataverseName dataverseName, String indexName) {
+        return new EntityDetails(databaseName, dataverseName, indexName, EntityType.INDEX);
+    }
+
+    public static EntityDetails newExtension(String extensionName) {
+        return new EntityDetails(null, null, extensionName, null);
+    }
+
     public String getDatabaseName() {
         return databaseName;
     }
@@ -55,4 +107,12 @@
     public EntityType getEntityType() {
         return entityType;
     }
+
+    public int getFunctionArity() {
+        return functionArity;
+    }
+
+    public static String getFunctionNameWithArity(String functionName, int functionArity) {
+        return functionName + "(" + functionArity + ")";
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Function.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Function.java
index 82c9514..3c1515b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Function.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Function.java
@@ -29,6 +29,7 @@
 import org.apache.asterix.common.metadata.DependencyFullyQualifiedName;
 import org.apache.asterix.metadata.MetadataCache;
 import org.apache.asterix.metadata.api.IMetadataEntity;
+import org.apache.asterix.metadata.utils.Creator;
 import org.apache.asterix.om.types.TypeSignature;
 
 public class Function implements IMetadataEntity<Function> {
@@ -49,12 +50,13 @@
     private final Boolean nullCall; // null for SQL++ and AQL functions
     private final Map<String, String> resources;
     private final List<List<DependencyFullyQualifiedName>> dependencies;
+    private final Creator creator;
 
     public Function(FunctionSignature signature, List<String> paramNames, List<TypeSignature> paramTypes,
             TypeSignature returnType, String functionBody, String functionKind, String language,
             String libraryDatabaseName, DataverseName libraryDataverseName, String libraryName,
             List<String> externalIdentifier, Boolean nullCall, Boolean deterministic, Map<String, String> resources,
-            List<List<DependencyFullyQualifiedName>> dependencies) {
+            List<List<DependencyFullyQualifiedName>> dependencies, Creator creator) {
         this.signature = signature;
         this.paramNames = paramNames;
         this.paramTypes = paramTypes;
@@ -72,6 +74,7 @@
         this.dependencies = dependencies == null
                 ? Arrays.asList(Collections.emptyList(), Collections.emptyList(), Collections.emptyList())
                 : dependencies;
+        this.creator = creator;
     }
 
     public FunctionSignature getSignature() {
@@ -161,6 +164,10 @@
         return dependencies;
     }
 
+    public Creator getCreator() {
+        return creator;
+    }
+
     @Override
     public Function addToCache(MetadataCache cache) {
         return cache.addFunctionIfNotExists(this);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
index 66192e0..4709c13 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
@@ -34,6 +34,7 @@
 import org.apache.asterix.common.transactions.IRecoveryManager.ResourceType;
 import org.apache.asterix.metadata.MetadataCache;
 import org.apache.asterix.metadata.api.IMetadataEntity;
+import org.apache.asterix.metadata.utils.Creator;
 import org.apache.asterix.metadata.utils.IndexUtil;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.AUnionType;
@@ -66,10 +67,11 @@
     private final boolean isEnforced;
     // Type of pending operations with respect to atomic DDL operation
     private int pendingOp;
+    private final Creator creator;
 
     public Index(String databaseName, DataverseName dataverseName, String datasetName, String indexName,
-            IndexType indexType, IIndexDetails indexDetails, boolean isEnforced, boolean isPrimaryIndex,
-            int pendingOp) {
+            IndexType indexType, IIndexDetails indexDetails, boolean isEnforced, boolean isPrimaryIndex, int pendingOp,
+            Creator creator) {
         boolean categoryOk = (indexType == null && indexDetails == null) || (IndexCategory
                 .of(Objects.requireNonNull(indexType)) == ((AbstractIndexDetails) Objects.requireNonNull(indexDetails))
                         .getIndexCategory());
@@ -85,26 +87,27 @@
         this.isPrimaryIndex = isPrimaryIndex;
         this.isEnforced = isEnforced;
         this.pendingOp = pendingOp;
+        this.creator = creator;
     }
 
     @Deprecated
     public Index(String database, DataverseName dataverseName, String datasetName, String indexName,
             IndexType indexType, List<List<String>> keyFieldNames, List<Integer> keyFieldSourceIndicators,
             List<IAType> keyFieldTypes, boolean overrideKeyFieldTypes, boolean isEnforced, boolean isPrimaryIndex,
-            int pendingOp, OptionalBoolean excludeUnknownKey) {
+            int pendingOp, OptionalBoolean excludeUnknownKey, Creator creator) {
         this(database, dataverseName, datasetName,
                 indexName, indexType, createSimpleIndexDetails(indexType, keyFieldNames, keyFieldSourceIndicators,
                         keyFieldTypes, overrideKeyFieldTypes, excludeUnknownKey),
-                isEnforced, isPrimaryIndex, pendingOp);
+                isEnforced, isPrimaryIndex, pendingOp, creator);
     }
 
     public static Index createPrimaryIndex(String database, DataverseName dataverseName, String datasetName,
             List<List<String>> keyFieldNames, List<Integer> keyFieldSourceIndicators, List<IAType> keyFieldTypes,
-            int pendingOp) {
-        return new Index(database, dataverseName, datasetName,
-                datasetName, IndexType.BTREE, new ValueIndexDetails(keyFieldNames, keyFieldSourceIndicators,
-                        keyFieldTypes, false, OptionalBoolean.empty(), OptionalBoolean.empty(), null, null, null),
-                false, true, pendingOp);
+            int pendingOp, Creator creator) {
+        return new Index(database, dataverseName, datasetName, datasetName, IndexType.BTREE,
+                new ValueIndexDetails(keyFieldNames, keyFieldSourceIndicators, keyFieldTypes, false,
+                        OptionalBoolean.empty(), OptionalBoolean.empty(), null, null, null),
+                false, true, pendingOp, creator);
     }
 
     public String getDatabaseName() {
@@ -155,6 +158,10 @@
         return !isPrimaryIndex();
     }
 
+    public Creator getCreator() {
+        return creator;
+    }
+
     public boolean isPrimaryKeyIndex() {
         // a primary key index has no key field names
         return indexType == IndexType.BTREE && ((ValueIndexDetails) indexDetails).keyFieldNames.isEmpty();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Synonym.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Synonym.java
index ebe780a..a355b6d 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Synonym.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Synonym.java
@@ -24,6 +24,7 @@
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.metadata.MetadataCache;
 import org.apache.asterix.metadata.api.IMetadataEntity;
+import org.apache.asterix.metadata.utils.Creator;
 
 public class Synonym implements IMetadataEntity<Synonym> {
 
@@ -40,15 +41,17 @@
     private final DataverseName objectDataverseName;
 
     private final String objectName;
+    private final Creator creator;
 
     public Synonym(String databaseName, DataverseName dataverseName, String synonymName, String objectDatabaseName,
-            DataverseName objectDataverseName, String objectName) {
+            DataverseName objectDataverseName, String objectName, Creator creator) {
         this.databaseName = Objects.requireNonNull(databaseName);
         this.dataverseName = Objects.requireNonNull(dataverseName);
         this.synonymName = Objects.requireNonNull(synonymName);
         this.objectDatabaseName = Objects.requireNonNull(objectDatabaseName);
         this.objectDataverseName = Objects.requireNonNull(objectDataverseName);
         this.objectName = Objects.requireNonNull(objectName);
+        this.creator = creator;
     }
 
     public String getDatabaseName() {
@@ -75,6 +78,10 @@
         return objectName;
     }
 
+    public Creator getCreator() {
+        return creator;
+    }
+
     @Override
     public boolean equals(Object o) {
         if (this == o)
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatabaseTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatabaseTupleTranslator.java
index 9752dc8..ef1b450 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatabaseTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatabaseTupleTranslator.java
@@ -21,13 +21,17 @@
 
 import java.util.Calendar;
 
+import org.apache.asterix.builders.RecordBuilder;
 import org.apache.asterix.metadata.bootstrap.DatabaseEntity;
+import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
 import org.apache.asterix.metadata.entities.Database;
+import org.apache.asterix.metadata.utils.Creator;
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.base.AMutableInt32;
 import org.apache.asterix.om.base.ARecord;
 import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -55,7 +59,9 @@
         boolean isSystemDatabase =
                 ((ABoolean) databaseRecord.getValueByPos(databaseEntity.systemDatabaseIndex())).getBoolean();
         int pendingOp = ((AInt32) databaseRecord.getValueByPos(databaseEntity.pendingOpIndex())).getIntegerValue();
-        return new Database(databaseName, isSystemDatabase, pendingOp);
+        Creator creator = Creator.createOrDefault(databaseRecord);
+
+        return new Database(databaseName, isSystemDatabase, pendingOp, creator);
     }
 
     @Override
@@ -94,6 +100,9 @@
         int32Serde.serialize(aInt32, fieldValue.getDataOutput());
         recordBuilder.addField(databaseEntity.pendingOpIndex(), fieldValue);
 
+        // write open fields
+        writeOpenFields(database);
+
         // write the payload record
         recordBuilder.write(tupleBuilder.getDataOutput(), true);
         tupleBuilder.addFieldEndOffset();
@@ -101,4 +110,39 @@
         tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
         return tuple;
     }
+
+    protected void writeOpenFields(Database database) throws HyracksDataException {
+        writeDatabaseCreator(database);
+    }
+
+    private void writeDatabaseCreator(Database database) throws HyracksDataException {
+        if (databaseEntity.databaseNameIndex() >= 0) {
+            Creator creatorInfo = database.getCreator();
+            RecordBuilder creatorObject = new RecordBuilder();
+            creatorObject.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+
+            fieldName.reset();
+            aString.setValue(MetadataRecordTypes.FIELD_NAME_CREATOR_NAME);
+            stringSerde.serialize(aString, fieldName.getDataOutput());
+            fieldValue.reset();
+            aString.setValue(creatorInfo.getName());
+            stringSerde.serialize(aString, fieldValue.getDataOutput());
+            creatorObject.addField(fieldName, fieldValue);
+
+            fieldName.reset();
+            aString.setValue(MetadataRecordTypes.FIELD_NAME_CREATOR_UUID);
+            stringSerde.serialize(aString, fieldName.getDataOutput());
+            fieldValue.reset();
+            aString.setValue(creatorInfo.getUuid());
+            stringSerde.serialize(aString, fieldValue.getDataOutput());
+            creatorObject.addField(fieldName, fieldValue);
+
+            fieldName.reset();
+            aString.setValue(MetadataRecordTypes.CREATOR_ARECORD_FIELD_NAME);
+            stringSerde.serialize(aString, fieldName.getDataOutput());
+            fieldValue.reset();
+            creatorObject.write(fieldValue.getDataOutput(), true);
+            recordBuilder.addField(fieldName, fieldValue);
+        }
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
index 7613dd3..191da69 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
@@ -52,6 +52,7 @@
 import org.apache.asterix.metadata.entities.InternalDatasetDetails.FileStructure;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails.PartitioningStrategy;
 import org.apache.asterix.metadata.entities.ViewDetails;
+import org.apache.asterix.metadata.utils.Creator;
 import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.base.ADateTime;
@@ -412,11 +413,12 @@
         long rebalanceCount = getRebalanceCount(datasetRecord);
         String compressionScheme = getCompressionScheme(datasetRecord);
         DatasetFormatInfo datasetFormatInfo = getDatasetFormatInfo(datasetRecord);
+        Creator creator = Creator.createOrDefault(datasetRecord);
 
         return new Dataset(databaseName, dataverseName, datasetName, itemTypeDatabaseName, typeDataverseName, typeName,
                 metaItemTypeDatabaseName, metaTypeDataverseName, metaTypeName, nodeGroupName, compactionPolicy.first,
                 compactionPolicy.second, datasetDetails, hints, datasetType, datasetId, pendingOp, rebalanceCount,
-                compressionScheme, datasetFormatInfo);
+                compressionScheme, datasetFormatInfo, creator);
     }
 
     protected Pair<String, Map<String, String>> readCompactionPolicy(DatasetType datasetType, ARecord datasetRecord) {
@@ -680,6 +682,7 @@
         writeBlockLevelStorageCompression(dataset);
         writeOpenDetails(dataset);
         writeDatasetFormatInfo(dataset);
+        writeDatasetCreator(dataset);
     }
 
     private void writeOpenDetails(Dataset dataset) throws HyracksDataException {
@@ -863,4 +866,35 @@
 
         propertyRecordBuilder.write(out, true);
     }
+
+    private void writeDatasetCreator(Dataset dataset) throws HyracksDataException {
+        if (datasetEntity.databaseNameIndex() >= 0) {
+            Creator creatorInfo = dataset.getCreator();
+            RecordBuilder creatorObject = new RecordBuilder();
+            creatorObject.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+
+            fieldName.reset();
+            aString.setValue(MetadataRecordTypes.FIELD_NAME_CREATOR_NAME);
+            stringSerde.serialize(aString, fieldName.getDataOutput());
+            fieldValue.reset();
+            aString.setValue(creatorInfo.getName());
+            stringSerde.serialize(aString, fieldValue.getDataOutput());
+            creatorObject.addField(fieldName, fieldValue);
+
+            fieldName.reset();
+            aString.setValue(MetadataRecordTypes.FIELD_NAME_CREATOR_UUID);
+            stringSerde.serialize(aString, fieldName.getDataOutput());
+            fieldValue.reset();
+            aString.setValue(creatorInfo.getUuid());
+            stringSerde.serialize(aString, fieldValue.getDataOutput());
+            creatorObject.addField(fieldName, fieldValue);
+
+            fieldName.reset();
+            aString.setValue(MetadataRecordTypes.CREATOR_ARECORD_FIELD_NAME);
+            stringSerde.serialize(aString, fieldName.getDataOutput());
+            fieldValue.reset();
+            creatorObject.write(fieldValue.getDataOutput(), true);
+            recordBuilder.addField(fieldName, fieldValue);
+        }
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java
index 42fcfbb..10c4b07 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java
@@ -21,14 +21,18 @@
 
 import java.util.Calendar;
 
+import org.apache.asterix.builders.RecordBuilder;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.metadata.MetadataUtil;
 import org.apache.asterix.metadata.bootstrap.DataverseEntity;
+import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
 import org.apache.asterix.metadata.entities.Dataverse;
+import org.apache.asterix.metadata.utils.Creator;
 import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.base.AMutableInt32;
 import org.apache.asterix.om.base.ARecord;
 import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -55,6 +59,7 @@
                 ((AString) dataverseRecord.getValueByPos(dataverseEntity.dataverseNameIndex())).getStringValue();
         DataverseName dataverseName = DataverseName.createFromCanonicalForm(dataverseCanonicalName);
         int databaseNameIndex = dataverseEntity.databaseNameIndex();
+
         String databaseName;
         if (databaseNameIndex >= 0) {
             databaseName = ((AString) dataverseRecord.getValueByPos(databaseNameIndex)).getStringValue();
@@ -63,8 +68,9 @@
         }
         String format = ((AString) dataverseRecord.getValueByPos(dataverseEntity.dataFormatIndex())).getStringValue();
         int pendingOp = ((AInt32) dataverseRecord.getValueByPos(dataverseEntity.pendingOpIndex())).getIntegerValue();
+        Creator creator = Creator.createOrDefault(dataverseRecord);
 
-        return new Dataverse(databaseName, dataverseName, format, pendingOp);
+        return new Dataverse(databaseName, dataverseName, format, pendingOp, creator);
     }
 
     @Override
@@ -115,6 +121,9 @@
         int32Serde.serialize(aInt32, fieldValue.getDataOutput());
         recordBuilder.addField(dataverseEntity.pendingOpIndex(), fieldValue);
 
+        // write open fields
+        writeOpenFields(dataverse);
+
         // write record
         recordBuilder.write(tupleBuilder.getDataOutput(), true);
         tupleBuilder.addFieldEndOffset();
@@ -122,4 +131,39 @@
         tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
         return tuple;
     }
+
+    protected void writeOpenFields(Dataverse dataverse) throws HyracksDataException {
+        writeDataverseCreator(dataverse);
+    }
+
+    private void writeDataverseCreator(Dataverse dataverse) throws HyracksDataException {
+        if (dataverseEntity.databaseNameIndex() >= 0) {
+            RecordBuilder creatorObject = new RecordBuilder();
+            creatorObject.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+            Creator creatorInfo = dataverse.getCreator();
+
+            fieldName.reset();
+            aString.setValue(MetadataRecordTypes.FIELD_NAME_CREATOR_NAME);
+            stringSerde.serialize(aString, fieldName.getDataOutput());
+            fieldValue.reset();
+            aString.setValue(creatorInfo.getName());
+            stringSerde.serialize(aString, fieldValue.getDataOutput());
+            creatorObject.addField(fieldName, fieldValue);
+
+            fieldName.reset();
+            aString.setValue(MetadataRecordTypes.FIELD_NAME_CREATOR_UUID);
+            stringSerde.serialize(aString, fieldName.getDataOutput());
+            fieldValue.reset();
+            aString.setValue(creatorInfo.getUuid());
+            stringSerde.serialize(aString, fieldValue.getDataOutput());
+            creatorObject.addField(fieldName, fieldValue);
+
+            fieldName.reset();
+            aString.setValue(MetadataRecordTypes.CREATOR_ARECORD_FIELD_NAME);
+            stringSerde.serialize(aString, fieldName.getDataOutput());
+            fieldValue.reset();
+            creatorObject.write(fieldValue.getDataOutput(), true);
+            recordBuilder.addField(fieldName, fieldValue);
+        }
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
index 8191212..2741d12 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
@@ -61,6 +61,7 @@
 import org.apache.asterix.metadata.bootstrap.FunctionEntity;
 import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
 import org.apache.asterix.metadata.entities.Function;
+import org.apache.asterix.metadata.utils.Creator;
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.base.AOrderedList;
@@ -203,10 +204,11 @@
         }
 
         FunctionSignature signature = new FunctionSignature(databaseName, dataverseName, functionName, arity);
+        Creator creator = Creator.createOrDefault(functionRecord);
 
         return new Function(signature, paramNames, paramTypes, returnType, definition, functionKind, language,
                 libraryDatabaseName, libraryDataverseName, libraryName, externalIdentifier, nullCall, deterministic,
-                resources, dependencies);
+                resources, dependencies, creator);
     }
 
     private List<TypeSignature> getParamTypes(ARecord functionRecord, String functionDatabaseName,
@@ -432,6 +434,7 @@
         writeLibrary(function);
         writeNullCall(function);
         writeDeterministic(function);
+        writeFunctionCreator(function);
     }
 
     protected void writeResources(Function function) throws HyracksDataException {
@@ -688,4 +691,35 @@
                 throw new AsterixException(ErrorCode.METADATA_ERROR, language);
         }
     }
+
+    private void writeFunctionCreator(Function function) throws HyracksDataException {
+        if (functionEntity.databaseNameIndex() >= 0) {
+            Creator creatorInfo = function.getCreator();
+            RecordBuilder creatorObject = new RecordBuilder();
+            creatorObject.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+
+            fieldName.reset();
+            aString.setValue(MetadataRecordTypes.FIELD_NAME_CREATOR_NAME);
+            stringSerde.serialize(aString, fieldName.getDataOutput());
+            fieldValue.reset();
+            aString.setValue(creatorInfo.getName());
+            stringSerde.serialize(aString, fieldValue.getDataOutput());
+            creatorObject.addField(fieldName, fieldValue);
+
+            fieldName.reset();
+            aString.setValue(MetadataRecordTypes.FIELD_NAME_CREATOR_UUID);
+            stringSerde.serialize(aString, fieldName.getDataOutput());
+            fieldValue.reset();
+            aString.setValue(creatorInfo.getUuid());
+            stringSerde.serialize(aString, fieldValue.getDataOutput());
+            creatorObject.addField(fieldName, fieldValue);
+
+            fieldName.reset();
+            aString.setValue(MetadataRecordTypes.CREATOR_ARECORD_FIELD_NAME);
+            stringSerde.serialize(aString, fieldName.getDataOutput());
+            fieldValue.reset();
+            creatorObject.write(fieldValue.getDataOutput(), true);
+            recordBuilder.addField(fieldName, fieldValue);
+        }
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
index 453ccc1..e726db7 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
@@ -42,10 +42,12 @@
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.metadata.MetadataNode;
 import org.apache.asterix.metadata.bootstrap.IndexEntity;
+import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
 import org.apache.asterix.metadata.declared.MetadataManagerUtil;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Datatype;
 import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.Creator;
 import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.base.ACollectionCursor;
@@ -60,6 +62,7 @@
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.base.IACursor;
 import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
 import org.apache.asterix.om.types.AOrderedListType;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
@@ -382,11 +385,18 @@
                         Triple<IAType, Boolean, Boolean> projectTypeResult =
                                 KeyFieldTypeUtil.getKeyProjectType((ARecordType) inputTypePrime, projectPath, null);
                         if (projectTypeResult == null) {
-                            throw new AsterixException(ErrorCode.METADATA_ERROR, projectPath.toString());
+                            if (indexType != IndexType.BTREE) {
+                                throw new AsterixException(ErrorCode.METADATA_ERROR, projectPath.toString());
+                            }
+                            projectTypePrime = BuiltinType.ANY;
+                            // We do not want the type to be union of Any, null and missing. Any will cover it all.
+                            projectTypeNullable = false;
+                            projectTypeMissable = false;
+                        } else {
+                            projectTypePrime = projectTypeResult.first;
+                            projectTypeNullable = inputTypeNullable || projectTypeResult.second;
+                            projectTypeMissable = inputTypeMissable || projectTypeResult.third;
                         }
-                        projectTypePrime = projectTypeResult.first;
-                        projectTypeNullable = inputTypeNullable || projectTypeResult.second;
-                        projectTypeMissable = inputTypeMissable || projectTypeResult.third;
                     }
                     IAType projectType = projectTypePrime == null ? null
                             : KeyFieldTypeUtil.makeUnknownableType(projectTypePrime, projectTypeNullable,
@@ -535,9 +545,10 @@
             isEnforcingKeys = ((ABoolean) indexRecord.getValueByPos(isEnforcedFieldPos)).getBoolean();
         }
         int pendingOp = ((AInt32) indexRecord.getValueByPos(indexEntity.pendingOpIndex())).getIntegerValue();
+        Creator creator = Creator.createOrDefault(indexRecord);
 
         return new Index(databaseName, dataverseName, datasetName, indexName, indexType, indexDetails, isEnforcingKeys,
-                isPrimaryIndex, pendingOp);
+                isPrimaryIndex, pendingOp, creator);
     }
 
     @Override
@@ -683,6 +694,7 @@
         writeExcludeUnknownKey(index);
         writeCast(index);
         writeSampleDetails(index);
+        writeIndexCreator(index);
     }
 
     private void writeComplexSearchKeys(Index.ArrayIndexDetails indexDetails) throws HyracksDataException {
@@ -1014,4 +1026,35 @@
             }
         }
     }
+
+    private void writeIndexCreator(Index index) throws HyracksDataException {
+        if (indexEntity.databaseNameIndex() >= 0) {
+            Creator creatorInfo = index.getCreator();
+            RecordBuilder creatorObject = new RecordBuilder();
+            creatorObject.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+
+            fieldName.reset();
+            aString.setValue(MetadataRecordTypes.FIELD_NAME_CREATOR_NAME);
+            stringSerde.serialize(aString, fieldName.getDataOutput());
+            fieldValue.reset();
+            aString.setValue(creatorInfo.getName());
+            stringSerde.serialize(aString, fieldValue.getDataOutput());
+            creatorObject.addField(fieldName, fieldValue);
+
+            fieldName.reset();
+            aString.setValue(MetadataRecordTypes.FIELD_NAME_CREATOR_UUID);
+            stringSerde.serialize(aString, fieldName.getDataOutput());
+            fieldValue.reset();
+            aString.setValue(creatorInfo.getUuid());
+            stringSerde.serialize(aString, fieldValue.getDataOutput());
+            creatorObject.addField(fieldName, fieldValue);
+
+            fieldName.reset();
+            aString.setValue(MetadataRecordTypes.CREATOR_ARECORD_FIELD_NAME);
+            stringSerde.serialize(aString, fieldName.getDataOutput());
+            fieldValue.reset();
+            creatorObject.write(fieldValue.getDataOutput(), true);
+            recordBuilder.addField(fieldName, fieldValue);
+        }
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/SynonymTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/SynonymTupleTranslator.java
index bb3ad28..389322c 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/SynonymTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/SynonymTupleTranslator.java
@@ -19,13 +19,16 @@
 
 package org.apache.asterix.metadata.entitytupletranslators;
 
+import org.apache.asterix.builders.RecordBuilder;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.metadata.MetadataUtil;
 import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
 import org.apache.asterix.metadata.bootstrap.SynonymEntity;
 import org.apache.asterix.metadata.entities.Synonym;
+import org.apache.asterix.metadata.utils.Creator;
 import org.apache.asterix.om.base.ARecord;
 import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -70,9 +73,10 @@
         }
 
         String objectName = ((AString) synonymRecord.getValueByPos(synonymEntity.objectNameIndex())).getStringValue();
+        Creator creator = Creator.createOrDefault(synonymRecord);
 
         return new Synonym(databaseName, dataverseName, synonymName, objectDatabaseName, objectDataverseName,
-                objectName);
+                objectName, creator);
     }
 
     @Override
@@ -149,5 +153,37 @@
             stringSerde.serialize(aString, fieldValue.getDataOutput());
             recordBuilder.addField(fieldName, fieldValue);
         }
+        writeSynonymCreator(synonym);
+    }
+
+    private void writeSynonymCreator(Synonym synonym) throws HyracksDataException {
+        if (synonymEntity.databaseNameIndex() >= 0) {
+            Creator creatorInfo = synonym.getCreator();
+            RecordBuilder creatorObject = new RecordBuilder();
+            creatorObject.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+
+            fieldName.reset();
+            aString.setValue(MetadataRecordTypes.FIELD_NAME_CREATOR_NAME);
+            stringSerde.serialize(aString, fieldName.getDataOutput());
+            fieldValue.reset();
+            aString.setValue(creatorInfo.getName());
+            stringSerde.serialize(aString, fieldValue.getDataOutput());
+            creatorObject.addField(fieldName, fieldValue);
+
+            fieldName.reset();
+            aString.setValue(MetadataRecordTypes.FIELD_NAME_CREATOR_UUID);
+            stringSerde.serialize(aString, fieldName.getDataOutput());
+            fieldValue.reset();
+            aString.setValue(creatorInfo.getUuid());
+            stringSerde.serialize(aString, fieldValue.getDataOutput());
+            creatorObject.addField(fieldName, fieldValue);
+
+            fieldName.reset();
+            aString.setValue(MetadataRecordTypes.CREATOR_ARECORD_FIELD_NAME);
+            stringSerde.serialize(aString, fieldName.getDataOutput());
+            fieldValue.reset();
+            creatorObject.write(fieldValue.getDataOutput(), true);
+            recordBuilder.addField(fieldName, fieldValue);
+        }
     }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/provider/ExternalWriterProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/provider/ExternalWriterProvider.java
index b8583d0..e6716df 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/provider/ExternalWriterProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/provider/ExternalWriterProvider.java
@@ -22,25 +22,50 @@
 import java.util.Map;
 import java.util.zip.Deflater;
 
+import org.apache.asterix.cloud.parquet.ParquetSinkExternalWriterFactory;
 import org.apache.asterix.cloud.writer.GCSExternalFileWriterFactory;
 import org.apache.asterix.cloud.writer.S3ExternalFileWriterFactory;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.writer.HDFSExternalFileWriterFactory;
 import org.apache.asterix.external.writer.LocalFSExternalFileWriterFactory;
 import org.apache.asterix.external.writer.compressor.GzipExternalFileCompressStreamFactory;
 import org.apache.asterix.external.writer.compressor.IExternalFileCompressStreamFactory;
 import org.apache.asterix.external.writer.compressor.NoOpExternalFileCompressStreamFactory;
+import org.apache.asterix.external.writer.printer.CsvExternalFilePrinterFactory;
+import org.apache.asterix.external.writer.printer.ParquetExternalFilePrinterFactory;
+import org.apache.asterix.external.writer.printer.ParquetExternalFilePrinterFactoryProvider;
 import org.apache.asterix.external.writer.printer.TextualExternalFilePrinterFactory;
+import org.apache.asterix.formats.nontagged.CSVPrinterFactoryProvider;
 import org.apache.asterix.formats.nontagged.CleanJSONPrinterFactoryProvider;
+import org.apache.asterix.metadata.declared.IExternalWriteDataSink;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.asterix.runtime.writer.ExternalFileWriterConfiguration;
+import org.apache.asterix.runtime.writer.ExternalFileWriterFactory;
 import org.apache.asterix.runtime.writer.IExternalFileWriterFactory;
 import org.apache.asterix.runtime.writer.IExternalFileWriterFactoryProvider;
 import org.apache.asterix.runtime.writer.IExternalPrinterFactory;
+import org.apache.asterix.runtime.writer.PathResolverFactory;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IWriteDataSink;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
+import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.operators.writer.SinkExternalWriterRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.writer.WriterPartitionerFactory;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.control.cc.ClusterControllerService;
+import org.apache.hyracks.util.StorageUtil;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
 
 public class ExternalWriterProvider {
     private static final Map<String, IExternalFileWriterFactoryProvider> CREATOR_MAP;
@@ -53,9 +78,10 @@
         addCreator(ExternalDataConstants.KEY_ADAPTER_NAME_LOCALFS, LocalFSExternalFileWriterFactory.PROVIDER);
         addCreator(ExternalDataConstants.KEY_ADAPTER_NAME_AWS_S3, S3ExternalFileWriterFactory.PROVIDER);
         addCreator(ExternalDataConstants.KEY_ADAPTER_NAME_GCS, GCSExternalFileWriterFactory.PROVIDER);
+        addCreator(ExternalDataConstants.KEY_ADAPTER_NAME_HDFS, HDFSExternalFileWriterFactory.PROVIDER);
     }
 
-    public static IExternalFileWriterFactory createWriterFactory(ICcApplicationContext appCtx, IWriteDataSink sink,
+    private static IExternalFileWriterFactory createWriterFactory(ICcApplicationContext appCtx, IWriteDataSink sink,
             String staticPath, SourceLocation pathExpressionLocation) {
         String adapterName = sink.getAdapterName().toLowerCase();
         IExternalFileWriterFactoryProvider creator = CREATOR_MAP.get(adapterName);
@@ -67,14 +93,14 @@
         return creator.create(createConfiguration(appCtx, sink, staticPath, pathExpressionLocation));
     }
 
-    public static String getFileExtension(IWriteDataSink sink) {
+    private static String getFileExtension(IWriteDataSink sink) {
         Map<String, String> configuration = sink.getConfiguration();
         String format = getFormat(configuration);
         String compression = getCompression(configuration);
         return format + (compression.isEmpty() ? "" : "." + compression);
     }
 
-    public static int getMaxResult(IWriteDataSink sink) {
+    private static int getMaxResult(IWriteDataSink sink) {
         String maxResultString = sink.getConfiguration().get(ExternalDataConstants.KEY_WRITER_MAX_RESULT);
         if (maxResultString == null) {
             return ExternalDataConstants.WRITER_MAX_RESULT_DEFAULT;
@@ -82,6 +108,14 @@
         return Integer.parseInt(maxResultString);
     }
 
+    private static int getMaxParquetSchema(Map<String, String> conf) {
+        String maxResultString = conf.get(ExternalDataConstants.PARQUET_MAX_SCHEMAS_KEY);
+        if (maxResultString == null) {
+            return ExternalDataConstants.PARQUET_MAX_SCHEMAS_DEFAULT_VALUE;
+        }
+        return Integer.parseInt(maxResultString);
+    }
+
     private static ExternalFileWriterConfiguration createConfiguration(ICcApplicationContext appCtx,
             IWriteDataSink sink, String staticPath, SourceLocation pathExpressionLocation) {
         Map<String, String> params = sink.getConfiguration();
@@ -106,21 +140,127 @@
         CREATOR_MAP.put(adapterName.toLowerCase(), creator);
     }
 
-    public static IExternalPrinterFactory createPrinter(ICcApplicationContext appCtx, IWriteDataSink sink,
-            Object sourceType) {
+    public static IPushRuntimeFactory getWriteFileRuntime(ICcApplicationContext appCtx, IWriteDataSink sink,
+            Object sourceType, ILogicalExpression staticPathExpr, SourceLocation pathSourceLocation,
+            IScalarEvaluatorFactory dynamicPathEvalFactory, RecordDescriptor inputDesc, int sourceColumn,
+            int[] partitionColumns, IBinaryComparatorFactory[] partitionComparatorFactories)
+            throws AlgebricksException {
+        String staticPath = staticPathExpr != null ? ConstantExpressionUtil.getStringConstant(staticPathExpr) : null;
+        IExternalFileWriterFactory fileWriterFactory =
+                ExternalWriterProvider.createWriterFactory(appCtx, sink, staticPath, pathSourceLocation);
+        fileWriterFactory.validate(appCtx);
+        String fileExtension = ExternalWriterProvider.getFileExtension(sink);
+        int maxResult = ExternalWriterProvider.getMaxResult(sink);
+
         Map<String, String> configuration = sink.getConfiguration();
         String format = configuration.get(ExternalDataConstants.KEY_FORMAT);
 
-        // Only JSON is supported for now
-        if (!ExternalDataConstants.FORMAT_JSON_LOWER_CASE.equalsIgnoreCase(format)) {
+        // Check for supported formats
+        if (!ExternalDataConstants.WRITER_SUPPORTED_FORMATS.contains(format.toLowerCase())) {
             throw new UnsupportedOperationException("Unsupported format " + format);
         }
 
         String compression = getCompression(configuration);
-        IExternalFileCompressStreamFactory compressStreamFactory =
-                createCompressionStreamFactory(appCtx, compression, configuration);
-        IPrinterFactory printerFactory = CleanJSONPrinterFactoryProvider.INSTANCE.getPrinterFactory(sourceType);
-        return new TextualExternalFilePrinterFactory(printerFactory, compressStreamFactory);
+        WriterPartitionerFactory partitionerFactory =
+                new WriterPartitionerFactory(partitionColumns, partitionComparatorFactories);
+        PathResolverFactory pathResolverFactory = new PathResolverFactory(fileWriterFactory, fileExtension,
+                dynamicPathEvalFactory, staticPath, pathSourceLocation);
+        IPrinterFactory printerFactory;
+        IExternalFileCompressStreamFactory compressStreamFactory;
+        IExternalPrinterFactory externalPrinterFactory;
+        ExternalFileWriterFactory writerFactory;
+        switch (format) {
+            case ExternalDataConstants.FORMAT_JSON_LOWER_CASE:
+                compressStreamFactory = createCompressionStreamFactory(appCtx, compression, configuration);
+                printerFactory = CleanJSONPrinterFactoryProvider.INSTANCE.getPrinterFactory(sourceType);
+                externalPrinterFactory = new TextualExternalFilePrinterFactory(printerFactory, compressStreamFactory);
+                writerFactory = new ExternalFileWriterFactory(fileWriterFactory, externalPrinterFactory,
+                        pathResolverFactory, maxResult);
+
+                return new SinkExternalWriterRuntimeFactory(sourceColumn, partitionColumns,
+                        partitionComparatorFactories, inputDesc, writerFactory);
+            case ExternalDataConstants.FORMAT_CSV_LOWER_CASE:
+                compressStreamFactory = createCompressionStreamFactory(appCtx, compression, configuration);
+                if (sink instanceof IExternalWriteDataSink) {
+                    ARecordType itemType = ((IExternalWriteDataSink) sink).getItemType();
+                    if (itemType != null) {
+                        printerFactory =
+                                CSVPrinterFactoryProvider
+                                        .createInstance(itemType, sink.getConfiguration(),
+                                                ((IExternalWriteDataSink) sink).getSourceLoc())
+                                        .getPrinterFactory(sourceType);
+                        externalPrinterFactory =
+                                new CsvExternalFilePrinterFactory(printerFactory, compressStreamFactory);
+                        writerFactory = new ExternalFileWriterFactory(fileWriterFactory, externalPrinterFactory,
+                                pathResolverFactory, maxResult);
+                        return new SinkExternalWriterRuntimeFactory(sourceColumn, partitionColumns,
+                                partitionComparatorFactories, inputDesc, writerFactory);
+                    } else {
+                        throw new CompilationException(ErrorCode.INVALID_CSV_SCHEMA);
+                    }
+                } else {
+                    throw new CompilationException(ErrorCode.INVALID_CSV_SCHEMA);
+                }
+
+            case ExternalDataConstants.FORMAT_PARQUET:
+
+                CompressionCodecName compressionCodecName;
+                if (compression == null || compression.equals("") || compression.equals("none")) {
+                    compressionCodecName = CompressionCodecName.UNCOMPRESSED;
+                } else {
+                    compressionCodecName = CompressionCodecName.valueOf(compression.toUpperCase());
+                }
+
+                String rowGroupSizeString = getRowGroupSize(configuration);
+                String pageSizeString = getPageSize(configuration);
+
+                long rowGroupSize = StorageUtil.getByteValue(rowGroupSizeString);
+                int pageSize = (int) StorageUtil.getByteValue(pageSizeString);
+                ParquetProperties.WriterVersion writerVersion = getParquetWriterVersion(configuration);
+
+                if (configuration.get(ExternalDataConstants.PARQUET_SCHEMA_KEY) != null) {
+                    String parquetSchemaString = configuration.get(ExternalDataConstants.PARQUET_SCHEMA_KEY);
+                    ParquetExternalFilePrinterFactory parquetPrinterFactory =
+                            new ParquetExternalFilePrinterFactory(compressionCodecName, parquetSchemaString,
+                                    (IAType) sourceType, rowGroupSize, pageSize, writerVersion);
+
+                    ExternalFileWriterFactory parquetWriterFactory = new ExternalFileWriterFactory(fileWriterFactory,
+                            parquetPrinterFactory, pathResolverFactory, maxResult);
+                    return new SinkExternalWriterRuntimeFactory(sourceColumn, partitionColumns,
+                            partitionComparatorFactories, inputDesc, parquetWriterFactory);
+                }
+
+                int maxSchemas = ExternalWriterProvider.getMaxParquetSchema(configuration);
+                ParquetExternalFilePrinterFactoryProvider printerFactoryProvider =
+                        new ParquetExternalFilePrinterFactoryProvider(compressionCodecName, (IAType) sourceType,
+                                rowGroupSize, pageSize, writerVersion);
+                return new ParquetSinkExternalWriterFactory(partitionerFactory, inputDesc, sourceColumn,
+                        (IAType) sourceType, maxSchemas, fileWriterFactory, maxResult, printerFactoryProvider,
+                        pathResolverFactory);
+
+            default:
+                throw new UnsupportedOperationException("Unsupported format " + format);
+        }
+
+    }
+
+    private static ParquetProperties.WriterVersion getParquetWriterVersion(Map<String, String> configuration) {
+        String writerVersionString = configuration.getOrDefault(ExternalDataConstants.PARQUET_WRITER_VERSION_KEY,
+                ExternalDataConstants.PARQUET_WRITER_VERSION_VALUE_1);
+        if (writerVersionString.equals(ExternalDataConstants.PARQUET_WRITER_VERSION_VALUE_2)) {
+            return ParquetProperties.WriterVersion.PARQUET_2_0;
+        }
+        return ParquetProperties.WriterVersion.PARQUET_1_0;
+    }
+
+    private static String getRowGroupSize(Map<String, String> configuration) {
+        return configuration.getOrDefault(ExternalDataConstants.KEY_PARQUET_ROW_GROUP_SIZE,
+                ExternalDataConstants.PARQUET_DEFAULT_ROW_GROUP_SIZE);
+    }
+
+    private static String getPageSize(Map<String, String> configuration) {
+        return configuration.getOrDefault(ExternalDataConstants.KEY_PARQUET_PAGE_SIZE,
+                ExternalDataConstants.PARQUET_DEFAULT_PAGE_SIZE);
     }
 
     private static String getFormat(Map<String, String> configuration) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/Creator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/Creator.java
new file mode 100644
index 0000000..49fca80f
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/Creator.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.metadata.utils;
+
+import java.io.Serializable;
+
+import org.apache.asterix.common.metadata.MetadataConstants;
+import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
+import org.apache.asterix.om.base.ARecord;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.types.ARecordType;
+
+public class Creator implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    public static final Creator DEFAULT_CREATOR =
+            new Creator(MetadataConstants.DEFAULT_CREATOR, MetadataConstants.DEFAULT_CREATOR_UUID);
+    private final String name;
+    private final String uuid;
+
+    private Creator(String name, String uuid) {
+        this.name = name;
+        this.uuid = uuid;
+    }
+
+    public static Creator create(String name, String uuid) {
+        return new Creator(name, uuid);
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public String getUuid() {
+        return uuid;
+    }
+
+    @Override
+    public String toString() {
+        return String.format("Creator{name='%s', uuid='%s'}", name, uuid);
+    }
+
+    public static Creator createOrDefault(ARecord record) {
+        ARecordType recType = record.getType();
+        int creatorIndex = recType.getFieldIndex(MetadataRecordTypes.CREATOR_ARECORD_FIELD_NAME);
+        String name = null, uuid = null;
+
+        if (creatorIndex >= 0) {
+            ARecordType creatorType = (ARecordType) recType.getFieldTypes()[creatorIndex];
+            ARecord creatorRecord = (ARecord) record.getValueByPos(creatorIndex);
+            int nameIndex = creatorType.getFieldIndex(MetadataRecordTypes.FIELD_NAME_CREATOR_NAME);
+            int uuidIndex = creatorType.getFieldIndex(MetadataRecordTypes.FIELD_NAME_CREATOR_UUID);
+
+            if (nameIndex >= 0) {
+                name = ((AString) creatorRecord.getValueByPos(nameIndex)).getStringValue();
+            }
+            if (uuidIndex >= 0) {
+                uuid = ((AString) creatorRecord.getValueByPos(uuidIndex)).getStringValue();
+            }
+            return create(name, uuid);
+        }
+        return DEFAULT_CREATOR;
+    }
+
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
index 67e4606..dda2111 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
@@ -94,7 +94,8 @@
     public static Index getPrimaryIndex(Dataset dataset) {
         InternalDatasetDetails id = (InternalDatasetDetails) dataset.getDatasetDetails();
         return Index.createPrimaryIndex(dataset.getDatabaseName(), dataset.getDataverseName(), dataset.getDatasetName(),
-                id.getPartitioningKey(), id.getKeySourceIndicator(), id.getPrimaryKeyType(), dataset.getPendingOp());
+                id.getPartitioningKey(), id.getKeySourceIndicator(), id.getPrimaryKeyType(), dataset.getPendingOp(),
+                dataset.getCreator());
     }
 
     public static int[] getBtreeFieldsIfFiltered(Dataset dataset, Index index) throws AlgebricksException {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
index 276f22b..cf4f8e0 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
@@ -49,6 +49,7 @@
 import org.apache.asterix.om.functions.IFunctionManager;
 import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
 import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.functions.AndDescriptor;
@@ -328,6 +329,10 @@
             throws AlgebricksException {
         IFunctionManager funManger = metadataProvider.getFunctionManager();
         IDataFormat dataFormat = metadataProvider.getDataFormat();
+        //if the target type is "BuiltinType.ANY" there is no need to cast. If not we have to cast.
+        if (ATypeTag.ANY.equals(targetType.getTypeTag())) {
+            return fieldEvalFactory;
+        }
 
         // check IndexUtil.castDefaultNull(index), too, because we always want to cast even if the overriding type is
         // the same as the overridden type (this is for the case where overriding the type of closed field is allowed)
diff --git a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java
index 2832470..788231c 100644
--- a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java
+++ b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java
@@ -32,6 +32,7 @@
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails.FileStructure;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails.PartitioningStrategy;
+import org.apache.asterix.metadata.utils.Creator;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.runtime.compression.CompressionManager;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -64,7 +65,7 @@
             Dataset dataset = new Dataset(db, dv, "log", itemTypeDb, itemTypeDv, "LogType", metaTypeDb, metaTypeDv,
                     "MetaType", "DEFAULT_NG_ALL_NODES", "prefix", compactionPolicyProperties, details,
                     Collections.emptyMap(), DatasetType.INTERNAL, 115, 0, CompressionManager.NONE,
-                    DatasetFormatInfo.SYSTEM_DEFAULT);
+                    DatasetFormatInfo.SYSTEM_DEFAULT, Creator.DEFAULT_CREATOR);
             DatasetTupleTranslator dtTranslator = new DatasetTupleTranslator(true, DatasetEntity.of(false));
             ITupleReference tuple = dtTranslator.getTupleFromMetadataEntity(dataset);
             Dataset deserializedDataset = dtTranslator.getMetadataEntityFromTuple(tuple);
diff --git a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
index 00f188d..713518d 100644
--- a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
+++ b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
@@ -41,6 +41,7 @@
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails.FileStructure;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails.PartitioningStrategy;
+import org.apache.asterix.metadata.utils.Creator;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
@@ -76,12 +77,13 @@
             Dataset dataset = new Dataset(dvTestDatabase, dvTest, "d1", itemTypeDatabase, dvFoo, "LogType",
                     metaTypeDatabase, dvCB, "MetaType", "DEFAULT_NG_ALL_NODES", "prefix", compactionPolicyProperties,
                     details, Collections.emptyMap(), DatasetType.INTERNAL, 115, 0, CompressionManager.NONE,
-                    DatasetFormatInfo.SYSTEM_DEFAULT);
+                    DatasetFormatInfo.SYSTEM_DEFAULT, Creator.DEFAULT_CREATOR);
 
             Index index = new Index(dvTestDatabase, dvTest, "d1", "i1", IndexType.BTREE,
                     Collections.singletonList(Collections.singletonList("row_id")),
                     indicator == null ? null : Collections.singletonList(indicator),
-                    Collections.singletonList(BuiltinType.AINT64), false, false, false, 0, OptionalBoolean.of(false));
+                    Collections.singletonList(BuiltinType.AINT64), false, false, false, 0, OptionalBoolean.of(false),
+                    null);
 
             MetadataNode mockMetadataNode = mock(MetadataNode.class);
             when(mockMetadataNode.getDatatype(any(), anyString(), any(DataverseName.class), anyString()))
diff --git a/asterixdb/asterix-om/pom.xml b/asterixdb/asterix-om/pom.xml
index 6db4840..9eccb86 100644
--- a/asterixdb/asterix-om/pom.xml
+++ b/asterixdb/asterix-om/pom.xml
@@ -143,8 +143,8 @@
       <artifactId>guava</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.esri.geometry</groupId>
-      <artifactId>esri-geometry-api</artifactId>
+      <groupId>org.locationtech.jts</groupId>
+      <artifactId>jts-core</artifactId>
     </dependency>
     <dependency>
       <groupId>javax.xml.bind</groupId>
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/TypeResolverUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/TypeResolverUtil.java
index 9d52a85..6d9007f 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/TypeResolverUtil.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/TypeResolverUtil.java
@@ -176,9 +176,9 @@
         IAType[] fieldTypesArray = fieldTypes.toArray(new IAType[0]);
         ARecordType resultType;
         if (resultTypeIsOpen && knowsAdditonalFieldNames) {
-            resultType = new ARecordType("generalized-record-type", fieldNamesArray, fieldTypesArray, resultTypeIsOpen,
-                    allPossibleAdditionalFieldNames);
             LinkedHashSet<String> resultFieldOrder = generalizeRecordFieldOrderHint(leftType, rightType);
+            resultType = new ARecordType("generalized-record-type", fieldNamesArray, fieldTypesArray, resultTypeIsOpen,
+                    allPossibleAdditionalFieldNames, resultFieldOrder);
             if (resultFieldOrder != null) {
                 resultType.getAnnotations().add(new RecordFieldOrderAnnotation(resultFieldOrder));
             }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AGeometryPartialBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AGeometryPartialBinaryComparatorFactory.java
index 57f9898..fa37722 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AGeometryPartialBinaryComparatorFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AGeometryPartialBinaryComparatorFactory.java
@@ -51,7 +51,7 @@
         if (c == 0) {
             AGeometry geometry = AGeometrySerializerDeserializer.getAGeometryObject(b1, s1);
             c = (geometry.getGeometry()
-                    .Equals(AGeometrySerializerDeserializer.getAGeometryObject(b2, s2).getGeometry())) ? 0 : 1;
+                    .equals(AGeometrySerializerDeserializer.getAGeometryObject(b2, s2).getGeometry())) ? 0 : 1;
         }
         return c;
     }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/PrintTools.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/PrintTools.java
index 449ce13..372b5fd 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/PrintTools.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/PrintTools.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers;
 
+import static org.apache.asterix.dataflow.data.nontagged.printers.csv.CSVUtils.DEFAULT_QUOTE;
+
 import java.io.DataOutput;
 import java.io.IOException;
 import java.io.OutputStream;
@@ -25,15 +27,21 @@
 import java.nio.charset.StandardCharsets;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.asterix.dataflow.data.nontagged.printers.csv.CSVUtils;
 import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.JtsModule;
 import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
 import org.apache.hyracks.algebricks.data.utils.WriteValueTools;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.util.bytes.HexPrinter;
 import org.apache.hyracks.util.string.UTF8StringUtil;
+import org.locationtech.jts.geom.Geometry;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
 
 public class PrintTools {
 
@@ -275,29 +283,55 @@
         UPPER_CASE,
     }
 
-    public static void writeUTF8StringAsCSV(byte[] b, int s, int l, OutputStream os) throws IOException {
+    public static void writeUTF8StringAsCSV(byte[] b, int s, int l, PrintStream ps, char quote, boolean forceQuote,
+            char escape, char delimiter) throws IOException {
         int stringLength = UTF8StringUtil.getUTFLength(b, s);
         int position = s + UTF8StringUtil.getNumBytesToStoreLength(stringLength);
         int maxPosition = position + stringLength;
-        os.write('"');
+        char quoteChar = quote == CSVUtils.NULL_CHAR ? DEFAULT_QUOTE : quote;
+
+        boolean shouldQuote = forceQuote;
+        if (!shouldQuote) {
+            // Check if the string contains any special characters that require quoting
+            for (int i = position; i < maxPosition; i++) {
+                char c = UTF8StringUtil.charAt(b, i);
+                if (c == quote || c == '\r' || c == '\n' || c == escape || c == delimiter) {
+                    shouldQuote = true;
+                    break;
+                }
+            }
+        }
+
+        if (shouldQuote) {
+            ps.print(quoteChar);
+        }
+
         while (position < maxPosition) {
             char c = UTF8StringUtil.charAt(b, position);
             int sz = UTF8StringUtil.charSize(b, position);
-            if (c == '"') {
-                os.write('"');
+
+            // todo: Escape character handling -- as the data is strictly quoted in case of carriage return, should "\r" needs to get handled?
+            if (c == quote || c == '\r') {
+                ps.print(escape);
             }
+
+            // Handling surrogate pairs
             if (Character.isHighSurrogate(c)) {
-                position += writeSupplementaryChar(os, b, maxPosition, position, c, sz);
+                position += writeSupplementaryChar(ps, b, maxPosition, position, c, sz);
                 continue;
             }
+
+            // Write the character bytes
             while (sz > 0) {
-                os.write(b[position]);
+                ps.print(c);
                 ++position;
                 --sz;
             }
-            break;
         }
-        os.write('"');
+
+        if (shouldQuote) {
+            ps.print(quoteChar);
+        }
     }
 
     public static void writeUTF8StringRaw(byte[] b, int s, int l, DataOutput os) throws IOException {
@@ -461,4 +495,23 @@
         return highSurrogateSize + lowSurrogateSize;
     }
 
+    /**
+     * Converts a JTS Geometry to a GeoJSON string. Returns Empty String in case of exceptions
+     *
+     * @param geometry The JTS Geometry to be converted.
+     * @return A GeoJSON string representation of the Geometry or an error message.
+     */
+    public static String geometryToGeoJSON(Geometry geometry) {
+        if (geometry == null) {
+            return "";
+        }
+        ObjectMapper mapper = new ObjectMapper();
+        mapper.registerModule(new JtsModule());
+        try {
+            return mapper.writeValueAsString(geometry);
+        } catch (JsonProcessingException e) {
+            return "";
+        }
+    }
+
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ANullPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ANullPrinterFactory.java
index dcf98a0..3d3ca3e 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ANullPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ANullPrinterFactory.java
@@ -19,16 +19,34 @@
 package org.apache.asterix.dataflow.data.nontagged.printers.csv;
 
 import java.io.PrintStream;
+import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.hyracks.algebricks.data.IPrinter;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
 
 public class ANullPrinterFactory implements IPrinterFactory {
-
     private static final long serialVersionUID = 1L;
-    public static final ANullPrinterFactory INSTANCE = new ANullPrinterFactory();
+    private static final String DEFAULT_NULL_STRING = "";
+    // Store the information about the instance based on the parameters
+    private static final ConcurrentHashMap<String, ANullPrinterFactory> instanceCache = new ConcurrentHashMap<>();
+    private String nullString;
 
-    public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> ps.print("null");
+    private ANullPrinterFactory(String nullString) {
+        this.nullString = nullString;
+    }
+
+    public static ANullPrinterFactory createInstance(String nullString) {
+        String key = CSVUtils.generateKey(nullString);
+        return instanceCache.computeIfAbsent(key, k -> new ANullPrinterFactory(nullString));
+    }
+
+    private final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
+        if (nullString != null) {
+            ps.print(nullString);
+        } else {
+            ps.print(DEFAULT_NULL_STRING);
+        }
+    };
 
     @Override
     public IPrinter createPrinter() {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/AObjectPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/AObjectPrinterFactory.java
index f1e2300..20b1ef0 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/AObjectPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/AObjectPrinterFactory.java
@@ -18,12 +18,22 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.csv;
 
+import static org.apache.asterix.dataflow.data.nontagged.printers.csv.CSVUtils.KEY_DELIMITER;
+import static org.apache.asterix.dataflow.data.nontagged.printers.csv.CSVUtils.KEY_EMPTY_FIELD_AS_NULL;
+import static org.apache.asterix.dataflow.data.nontagged.printers.csv.CSVUtils.KEY_ESCAPE;
+import static org.apache.asterix.dataflow.data.nontagged.printers.csv.CSVUtils.KEY_FORCE_QUOTE;
+import static org.apache.asterix.dataflow.data.nontagged.printers.csv.CSVUtils.KEY_NULL;
+import static org.apache.asterix.dataflow.data.nontagged.printers.csv.CSVUtils.KEY_QUOTE;
+
 import java.io.PrintStream;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.asterix.om.pointables.ARecordVisitablePointable;
 import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
 import org.apache.asterix.om.pointables.printer.IPrintVisitor;
 import org.apache.asterix.om.pointables.printer.csv.APrintVisitor;
+import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -32,11 +42,26 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class AObjectPrinterFactory implements IPrinterFactory {
-
     private static final long serialVersionUID = 1L;
-    public static final AObjectPrinterFactory INSTANCE = new AObjectPrinterFactory();
+    private static final ConcurrentHashMap<String, AObjectPrinterFactory> instanceCache = new ConcurrentHashMap<>();
+    private ARecordType itemType;
+    private Map<String, String> configuration;
+    private boolean emptyFieldAsNull;
 
-    public static boolean printFlatValue(ATypeTag typeTag, byte[] b, int s, int l, PrintStream ps)
+    private AObjectPrinterFactory(ARecordType itemType, Map<String, String> configuration) {
+        this.itemType = itemType;
+        this.configuration = configuration;
+        String emptyFieldAsNullStr = configuration.get(KEY_EMPTY_FIELD_AS_NULL);
+        this.emptyFieldAsNull = emptyFieldAsNullStr != null && Boolean.parseBoolean(emptyFieldAsNullStr);
+    }
+
+    public static AObjectPrinterFactory createInstance(ARecordType itemType, Map<String, String> configuration) {
+        // generate a unique identifier based on the parameters and hash the instance corresponding to it.
+        String key = CSVUtils.generateKey(itemType, configuration);
+        return instanceCache.computeIfAbsent(key, k -> new AObjectPrinterFactory(itemType, configuration));
+    }
+
+    public boolean printFlatValue(ATypeTag typeTag, byte[] b, int s, int l, PrintStream ps)
             throws HyracksDataException {
         switch (typeTag) {
             case TINYINT:
@@ -53,7 +78,7 @@
                 return true;
             case MISSING:
             case NULL:
-                ANullPrinterFactory.PRINTER.print(b, s, l, ps);
+                ANullPrinterFactory.createInstance(configuration.get(KEY_NULL)).createPrinter().print(b, s, l, ps);
                 return true;
             case BOOLEAN:
                 ABooleanPrinterFactory.PRINTER.print(b, s, l, ps);
@@ -104,7 +129,14 @@
                 ARectanglePrinterFactory.PRINTER.print(b, s, l, ps);
                 return true;
             case STRING:
-                AStringPrinterFactory.PRINTER.print(b, s, l, ps);
+                if (emptyFieldAsNull && CSVUtils.isEmptyString(b, s, l)) {
+                    ANullPrinterFactory.createInstance(configuration.get(KEY_NULL)).createPrinter().print(b, s, l, ps);
+                } else {
+                    AStringPrinterFactory
+                            .createInstance(configuration.get(KEY_QUOTE), configuration.get(KEY_FORCE_QUOTE),
+                                    configuration.get(KEY_ESCAPE), configuration.get(KEY_DELIMITER))
+                            .createPrinter().print(b, s, l, ps);
+                }
                 return true;
             case BINARY:
                 ABinaryHexPrinterFactory.PRINTER.print(b, s, l, ps);
@@ -119,11 +151,10 @@
 
     @Override
     public IPrinter createPrinter() {
-        final ARecordVisitablePointable rPointable =
+        final ARecordVisitablePointable recordVisitablePointable =
                 new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
         final Pair<PrintStream, ATypeTag> streamTag = new Pair<>(null, null);
-
-        final IPrintVisitor visitor = new APrintVisitor();
+        final IPrintVisitor visitor = new APrintVisitor(itemType, configuration);
 
         return (byte[] b, int s, int l, PrintStream ps) -> {
             ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(b[s]);
@@ -132,8 +163,8 @@
                 streamTag.second = typeTag;
                 switch (typeTag) {
                     case OBJECT:
-                        rPointable.set(b, s, l);
-                        visitor.visit(rPointable, streamTag);
+                        recordVisitablePointable.set(b, s, l);
+                        visitor.visit(recordVisitablePointable, streamTag);
                         break;
                     default:
                         throw new HyracksDataException("No printer for type " + typeTag);
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ARecordPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ARecordPrinterFactory.java
index 909fd60..2b31fb0 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ARecordPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ARecordPrinterFactory.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.dataflow.data.nontagged.printers.csv;
 
 import java.io.PrintStream;
+import java.util.Map;
 
 import org.apache.asterix.om.pointables.PointableAllocator;
 import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
@@ -36,9 +37,13 @@
 
     private static final long serialVersionUID = 1L;
     private final ARecordType recType;
+    private final ARecordType itemType;
+    private final Map<String, String> configuration;
 
-    public ARecordPrinterFactory(ARecordType recType) {
+    public ARecordPrinterFactory(ARecordType recType, ARecordType itemType, Map<String, String> configuration) {
         this.recType = recType;
+        this.itemType = itemType;
+        this.configuration = configuration;
     }
 
     @Override
@@ -47,7 +52,7 @@
         final IAType inputType =
                 recType == null ? DefaultOpenFieldType.getDefaultOpenFieldType(ATypeTag.OBJECT) : recType;
         final IVisitablePointable recAccessor = allocator.allocateRecordValue(inputType);
-        final APrintVisitor printVisitor = new APrintVisitor();
+        final APrintVisitor printVisitor = new APrintVisitor(itemType, configuration);
         final Pair<PrintStream, ATypeTag> arg = new Pair<>(null, null);
 
         return new IPrinter() {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/AStringPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/AStringPrinterFactory.java
index c217203..ae368bd 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/AStringPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/AStringPrinterFactory.java
@@ -18,8 +18,14 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.csv;
 
+import static org.apache.asterix.dataflow.data.nontagged.printers.csv.CSVUtils.DEFAULT_VALUES;
+import static org.apache.asterix.dataflow.data.nontagged.printers.csv.CSVUtils.KEY_DELIMITER;
+import static org.apache.asterix.dataflow.data.nontagged.printers.csv.CSVUtils.KEY_ESCAPE;
+import static org.apache.asterix.dataflow.data.nontagged.printers.csv.CSVUtils.KEY_QUOTE;
+
 import java.io.IOException;
 import java.io.PrintStream;
+import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.asterix.dataflow.data.nontagged.printers.PrintTools;
 import org.apache.hyracks.algebricks.data.IPrinter;
@@ -27,18 +33,52 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class AStringPrinterFactory implements IPrinterFactory {
-
     private static final long serialVersionUID = 1L;
-    public static final AStringPrinterFactory INSTANCE = new AStringPrinterFactory();
+    private static final ConcurrentHashMap<String, AStringPrinterFactory> instanceCache = new ConcurrentHashMap<>();
+    private static final String NONE = "none";
+    private String quote;
+    private Boolean forceQuote;
+    private String escape;
+    private String delimiter;
 
-    public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
+    private AStringPrinterFactory(String quote, Boolean forceQuote, String escape, String delimiter) {
+        this.quote = quote;
+        this.forceQuote = forceQuote;
+        this.escape = escape;
+        this.delimiter = delimiter;
+    }
+
+    public static AStringPrinterFactory createInstance(String quote, String forceQuoteStr, String escape,
+            String delimiter) {
+        boolean forceQuote = forceQuoteStr == null || Boolean.parseBoolean(forceQuoteStr);
+        String key = CSVUtils.generateKey(quote, forceQuoteStr, escape, delimiter);
+        return instanceCache.computeIfAbsent(key, k -> new AStringPrinterFactory(quote, forceQuote, escape, delimiter));
+    }
+
+    private final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
         try {
-            PrintTools.writeUTF8StringAsCSV(b, s + 1, l - 1, ps);
+            char quoteChar =
+                    quote == null ? extractSingleChar(DEFAULT_VALUES.get(KEY_QUOTE)) : extractSingleChar(quote);
+            char escapeChar =
+                    escape == null ? extractSingleChar(DEFAULT_VALUES.get(KEY_ESCAPE)) : extractSingleChar(escape);
+            char delimiterChar = delimiter == null ? extractSingleChar(DEFAULT_VALUES.get(KEY_DELIMITER))
+                    : extractSingleChar(delimiter);
+            PrintTools.writeUTF8StringAsCSV(b, s + 1, l - 1, ps, quoteChar, forceQuote, escapeChar, delimiterChar);
         } catch (IOException e) {
             throw HyracksDataException.create(e);
         }
     };
 
+    public char extractSingleChar(String input) throws IOException {
+        if (input != null && input.length() == 1) {
+            return input.charAt(0);
+        } else if (input.equalsIgnoreCase(NONE)) {
+            return CSVUtils.NULL_CHAR; // Replace 'none' with null character
+        } else {
+            throw new IOException("Input string must be a single character");
+        }
+    }
+
     @Override
     public IPrinter createPrinter() {
         return PRINTER;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/CSVUtils.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/CSVUtils.java
new file mode 100644
index 0000000..b50816a
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/CSVUtils.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.dataflow.data.nontagged.printers.csv;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.om.types.ARecordType;
+
+public class CSVUtils {
+
+    // Constants for the supported CSV parameters
+    public static final String KEY_NULL = "null";
+    public static final String KEY_ESCAPE = "escape";
+    public static final String KEY_HEADER = "header";
+    public static final String KEY_DELIMITER = "delimiter";
+    public static final String KEY_RECORD_DELIMITER = "recordDelimiter";
+    public static final String KEY_FORCE_QUOTE = "forceQuote";
+    public static final String KEY_QUOTE = "quote";
+    public static final String KEY_EMPTY_FIELD_AS_NULL = "empty_field_as_null";
+    public static final char DEFAULT_QUOTE = '"';
+    private static final String DEFAULT_DELIMITER_VALUE = ",";
+    private static final String DEFAULT_NULL_VALUE = "";
+    private static final String DOUBLE_QUOTES = "\"";
+    public static final char NULL_CHAR = '\0';
+    private static final String FALSE = "false";
+    private static final String DEFAULT_RECORD_DELIMITER = "\n";
+
+    // List of supported parameters
+    public static final List<String> CSV_PARAMETERS = Arrays.asList(KEY_NULL, KEY_ESCAPE, KEY_HEADER, KEY_DELIMITER,
+            KEY_RECORD_DELIMITER, KEY_FORCE_QUOTE, KEY_QUOTE, KEY_EMPTY_FIELD_AS_NULL);
+
+    // Default values for each parameter
+    public static final Map<String, String> DEFAULT_VALUES;
+
+    static {
+        DEFAULT_VALUES = new HashMap<>();
+        DEFAULT_VALUES.put(KEY_NULL, DEFAULT_NULL_VALUE);
+        DEFAULT_VALUES.put(KEY_ESCAPE, DOUBLE_QUOTES);
+        DEFAULT_VALUES.put(KEY_HEADER, FALSE);
+        DEFAULT_VALUES.put(KEY_DELIMITER, DEFAULT_DELIMITER_VALUE);
+        DEFAULT_VALUES.put(KEY_RECORD_DELIMITER, DEFAULT_RECORD_DELIMITER);
+        DEFAULT_VALUES.put(KEY_FORCE_QUOTE, FALSE);
+        DEFAULT_VALUES.put(KEY_QUOTE, DOUBLE_QUOTES);
+        DEFAULT_VALUES.put(KEY_EMPTY_FIELD_AS_NULL, FALSE);
+    }
+
+    // Generate a key based on configuration for ARecordType and parameters
+    public static String generateKey(ARecordType itemType, Map<String, String> configuration) {
+        StringBuilder keyBuilder = new StringBuilder();
+        keyBuilder.append(itemType == null ? KEY_NULL : itemType.toString()).append(" | ");
+        // Iterate through supported CSV parameters and append their values from configuration
+        for (String param : CSV_PARAMETERS) {
+            String value = configuration.getOrDefault(param, DEFAULT_VALUES.get(param));
+            keyBuilder.append(param).append(" : ").append(value).append(" | ");
+        }
+        // Remove the trailing " | "
+        if (keyBuilder.length() > 0 && keyBuilder.charAt(keyBuilder.length() - 2) == '|') {
+            keyBuilder.setLength(keyBuilder.length() - 3);
+        }
+        return keyBuilder.toString();
+    }
+
+    public static String generateKey(String quote, String forceQuoteStr, String escape, String delimiter) {
+        // Use default values when no values are specified (null)
+        return KEY_QUOTE + " : " + (quote != null ? quote : DEFAULT_VALUES.get(KEY_QUOTE)) + " | " + KEY_FORCE_QUOTE
+                + " : " + (forceQuoteStr != null ? forceQuoteStr : DEFAULT_VALUES.get(KEY_FORCE_QUOTE)) + " | "
+                + KEY_ESCAPE + " : " + (escape != null ? escape : DEFAULT_VALUES.get(KEY_ESCAPE)) + " | "
+                + KEY_DELIMITER + " : " + (delimiter != null ? delimiter : DEFAULT_VALUES.get(KEY_DELIMITER));
+    }
+
+    public static String generateKey(String nullString) {
+        // Use the default value when nullString is not specified (null)
+        return KEY_NULL + " : " + (nullString != null ? nullString : DEFAULT_VALUES.get(KEY_NULL));
+    }
+
+    public static boolean isEmptyString(byte[] b, int s, int l) {
+        return b == null || l <= 2 || s < 0 || s + l > b.length;
+    }
+
+    public static String getDelimiter(Map<String, String> configuration) {
+        return configuration.get(KEY_DELIMITER) == null ? DEFAULT_DELIMITER_VALUE : configuration.get(KEY_DELIMITER);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AGeometryPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AGeometryPrinterFactory.java
index cc74f85..848e1bd 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AGeometryPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AGeometryPrinterFactory.java
@@ -23,11 +23,11 @@
 import java.io.DataInputStream;
 import java.io.PrintStream;
 
+import org.apache.asterix.dataflow.data.nontagged.printers.PrintTools;
 import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
 import org.apache.hyracks.algebricks.data.IPrinter;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
-
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
 
 public class AGeometryPrinterFactory implements IPrinterFactory {
 
@@ -37,8 +37,8 @@
     public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
         ByteArrayInputStream inStream = new ByteArrayInputStream(b, s + 1, l - 1);
         DataInput dataIn = new DataInputStream(inStream);
-        OGCGeometry geometry = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn).getGeometry();
-        ps.print(geometry.asGeoJson());
+        Geometry geometry = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn).getGeometry();
+        ps.print(PrintTools.geometryToGeoJSON(geometry));
     };
 
     @Override
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AGeometrySerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AGeometrySerializerDeserializer.java
index 38a4c46..28c171d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AGeometrySerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AGeometrySerializerDeserializer.java
@@ -21,19 +21,18 @@
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.Arrays;
 
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.GeoFunctionUtils;
 import org.apache.asterix.om.base.AGeometry;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.esri.core.geometry.OperatorImportFromWkb;
-import com.esri.core.geometry.SpatialReference;
-import com.esri.core.geometry.WkbImportFlags;
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.io.ParseException;
+import org.locationtech.jts.io.WKBReader;
+import org.locationtech.jts.io.WKBWriter;
 
 public class AGeometrySerializerDeserializer implements ISerializerDeserializer<AGeometry> {
 
@@ -41,24 +40,19 @@
 
     public static final AGeometrySerializerDeserializer INSTANCE = new AGeometrySerializerDeserializer();
 
-    /**Use WGS 84 (EPSG:4326) as the default coordinate reference system*/
-    public static final SpatialReference DEFAULT_CRS = SpatialReference.create(4326);
-
     private AGeometrySerializerDeserializer() {
     }
 
     @Override
     public AGeometry deserialize(DataInput in) throws HyracksDataException {
+        WKBReader wkbReader = new WKBReader();
         try {
             int length = in.readInt();
             byte[] bytes = new byte[length];
             in.readFully(bytes);
-            ByteBuffer buffer = ByteBuffer.wrap(bytes);
-            OGCGeometry geometry = OGCGeometry.createFromOGCStructure(
-                    OperatorImportFromWkb.local().executeOGC(WkbImportFlags.wkbImportDefaults, buffer, null),
-                    DEFAULT_CRS);
+            Geometry geometry = wkbReader.read(bytes);
             return new AGeometry(geometry);
-        } catch (IOException e) {
+        } catch (IOException | ParseException e) {
             throw HyracksDataException.create(e);
         }
     }
@@ -66,8 +60,10 @@
     @Override
     public void serialize(AGeometry instance, DataOutput out) throws HyracksDataException {
         try {
-            OGCGeometry geometry = instance.getGeometry();
-            byte[] buffer = geometry.asBinary().array();
+            Geometry geometry = instance.getGeometry();
+            WKBWriter wkbWriter = new WKBWriter(GeoFunctionUtils.getCoordinateDimension(geometry),
+                    GeoFunctionUtils.LITTLE_ENDIAN_BYTEORDER);
+            byte[] buffer = wkbWriter.write(geometry);
             // For efficiency, we store the size of the geometry in bytes in the first 32 bits
             // This allows AsterixDB to skip over this attribute if needed.
             out.writeInt(buffer.length);
@@ -77,24 +73,40 @@
         }
     }
 
-    public static int getAGeometrySizeOffset() throws HyracksDataException {
+    public void serialize(Geometry geometry, DataOutput out) throws HyracksDataException {
+        try {
+            WKBWriter wkbWriter = new WKBWriter(GeoFunctionUtils.getCoordinateDimension(geometry),
+                    GeoFunctionUtils.LITTLE_ENDIAN_BYTEORDER);
+            byte[] buffer = wkbWriter.write(geometry);
+            // For efficiency, we store the size of the geometry in bytes in the first 32 bits
+            // This allows AsterixDB to skip over this attribute if needed.
+            out.writeInt(buffer.length);
+            out.write(buffer);
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    public static int getAGeometrySizeOffset() {
         return 0;
     }
 
     public static AGeometry getAGeometryObject(byte[] bytes, int startOffset) throws HyracksDataException {
         // Size of the AGeometry object is stored in bytes in the first 32 bits
         // See serialize method
+        WKBReader wkbReader = new WKBReader();
         int size = AInt32SerializerDeserializer.getInt(bytes, startOffset);
 
         if (bytes.length < startOffset + size + 4)
             // TODO(mmahin): this error code takes 5 parameters, and this is passing none, so I suspect this isn't right
             throw RuntimeDataException.create(ErrorCode.VALUE_OUT_OF_RANGE);
-
-        // Skip the size of the geometry in first 4 bytes
-        byte[] bytes1 = Arrays.copyOfRange(bytes, startOffset + 4, startOffset + size + 4);
-        ByteBuffer buffer = ByteBuffer.wrap(bytes1);
-        OGCGeometry geometry = OGCGeometry.createFromOGCStructure(
-                OperatorImportFromWkb.local().executeOGC(WkbImportFlags.wkbImportDefaults, buffer, null), DEFAULT_CRS);
-        return new AGeometry(geometry);
+        try {
+            // Skip the size of the geometry in first 4 bytes
+            byte[] bytes1 = Arrays.copyOfRange(bytes, startOffset + 4, startOffset + size + 4);
+            Geometry geometry = wkbReader.read(bytes1);
+            return new AGeometry(geometry);
+        } catch (ParseException e) {
+            throw HyracksDataException.create(e);
+        }
     }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/GeoFunctionUtils.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/GeoFunctionUtils.java
new file mode 100644
index 0000000..1e92b62
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/GeoFunctionUtils.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts;
+
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.CoordinateXYZM;
+import org.locationtech.jts.geom.Geometry;
+
+public class GeoFunctionUtils {
+    public static final int LITTLE_ENDIAN_BYTEORDER = 2;
+
+    /**
+     * Returns the dimension of the coordinate based on whether Z or M is defined.
+     * TODO: Add 4th dimension support
+     * @param geometry The geometry to check.
+     * @return the dimensionality of the coordinate (2 or 3 or 4).
+     */
+    public static int getCoordinateDimension(Geometry geometry) {
+        int dimension = 2;
+        if (geometry == null || geometry.isEmpty()) {
+            return 2;
+        }
+        Coordinate sample = geometry.getCoordinates()[0];
+        if (!Double.isNaN(sample.getZ())) {
+            dimension++;
+        }
+
+        if (sample instanceof CoordinateXYZM) {
+            CoordinateXYZM firstCoordXYZM = (CoordinateXYZM) sample;
+            if (!Double.isNaN(firstCoordXYZM.getM())) {
+                dimension = 3;
+            }
+        }
+        return dimension;
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/GeoJsonConstants.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/GeoJsonConstants.java
new file mode 100644
index 0000000..7f3d1de
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/GeoJsonConstants.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts;
+
+/**
+ * Constants used for GeoJSON serialization and deserialization of JTS geometries.
+ * This class provides string constants that represent various GeoJSON object types
+ * and properties, such as types of geometries and common GeoJSON attributes like
+ * coordinates and type.
+ * <p>
+ * These constants are used throughout the Jackson JTS module to ensure consistency
+ * in processing and generating GeoJSON.
+ */
+public class GeoJsonConstants {
+    public static final String POINT = "Point";
+    public static final String LINE_STRING = "LineString";
+    public static final String POLYGON = "Polygon";
+
+    public static final String MULTI_POINT = "MultiPoint";
+    public static final String MULTI_LINE_STRING = "MultiLineString";
+    public static final String MULTI_POLYGON = "MultiPolygon";
+
+    public static final String GEOMETRY_COLLECTION = "GeometryCollection";
+
+    public static final String TYPE = "type";
+
+    public static final String GEOMETRIES = "geometries";
+
+    public static final String COORDINATES = "coordinates";
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/JtsModule.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/JtsModule.java
new file mode 100644
index 0000000..77b31ce
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/JtsModule.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * This file includes code derived from the project "jackson-datatype-jts"
+ * under the Apache License 2.0.
+ * Original source: https://github.com/bedatadriven/jackson-datatype-jts/blob/master/src/main/java/com/bedatadriven/jackson/datatype/jts/JtsModule.java.
+ *
+ * Modifications:
+ * - Adapted the code to support the org.locationtech.jts package instead of com.vividsolutions.jts
+ *
+ * The modified version retains the original license and notices. For more information
+ * on the original project and licensing, please visit https://github.com/bedatadriven/jackson-datatype-jts.
+ */
+package org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.parsers.GenericGeometryParser;
+import org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.parsers.GeometryCollectionParser;
+import org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.parsers.LineStringParser;
+import org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.parsers.MultiLineStringParser;
+import org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.parsers.MultiPointParser;
+import org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.parsers.MultiPolygonParser;
+import org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.parsers.PointParser;
+import org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.parsers.PolygonParser;
+import org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.serde.GeometryDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.serde.GeometrySerializer;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryCollection;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.MultiLineString;
+import org.locationtech.jts.geom.MultiPoint;
+import org.locationtech.jts.geom.MultiPolygon;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+
+import com.fasterxml.jackson.core.Version;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+
+/**
+ * A Jackson module for serializing and deserializing JTS (Java Topology Suite) geometry objects.
+ * This module provides custom serializers and deserializers capable of handling various types
+ * of geometries such as Points, LineStrings, Polygons, and their respective multi-part counterparts,
+ * as well as GeometryCollections.
+ * <p>
+ * It leverages a {@link GeometryFactory} for the creation of geometry objects during deserialization,
+ * ensuring that geometry objects are constructed appropriately.
+ */
+public class JtsModule extends SimpleModule {
+    private static final long serialVersionUID = 324082011931609589L;
+
+    public JtsModule() {
+        this(new GeometryFactory());
+    }
+
+    /**
+     * Constructs a JtsModule with a specified {@link GeometryFactory}.
+     * This constructor allows for customization of the geometry factory used for creating
+     * JTS geometry objects, providing flexibility for various precision and srid settings.
+     *
+     * @param geometryFactory the geometry factory to use for creating geometry objects
+     *                        during deserialization
+     */
+    public JtsModule(GeometryFactory geometryFactory) {
+        super("JtsModule", new Version(1, 0, 0, null, null, null));
+
+        addSerializer(Geometry.class, new GeometrySerializer());
+        GenericGeometryParser genericGeometryParser = new GenericGeometryParser(geometryFactory);
+        addDeserializer(Geometry.class, new GeometryDeserializer<>(genericGeometryParser));
+        addDeserializer(Point.class, new GeometryDeserializer<>(new PointParser(geometryFactory)));
+        addDeserializer(MultiPoint.class, new GeometryDeserializer<>(new MultiPointParser(geometryFactory)));
+        addDeserializer(LineString.class, new GeometryDeserializer<>(new LineStringParser(geometryFactory)));
+        addDeserializer(MultiLineString.class, new GeometryDeserializer<>(new MultiLineStringParser(geometryFactory)));
+        addDeserializer(Polygon.class, new GeometryDeserializer<>(new PolygonParser(geometryFactory)));
+        addDeserializer(MultiPolygon.class, new GeometryDeserializer<>(new MultiPolygonParser(geometryFactory)));
+        addDeserializer(GeometryCollection.class,
+                new GeometryDeserializer<>(new GeometryCollectionParser(geometryFactory, genericGeometryParser)));
+    }
+
+    @Override
+    public void setupModule(SetupContext context) {
+        super.setupModule(context);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/BaseParser.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/BaseParser.java
new file mode 100644
index 0000000..4721680
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/BaseParser.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * This file includes code derived from the project "jackson-datatype-jts"
+ * under the Apache License 2.0.
+ * Original source: https://github.com/bedatadriven/jackson-datatype-jts/blob/master/src/main/java/com/bedatadriven/jackson/datatype/jts/parsers/BaseParser.java.
+ *
+ * Modifications:
+ * - Adapted the code to support the org.locationtech.jts package instead of com.vividsolutions.jts
+ *
+ * The modified version retains the original license and notices. For more information
+ * on the original project and licensing, please visit https://github.com/bedatadriven/jackson-datatype-jts.
+ */
+package org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.parsers;
+
+import org.locationtech.jts.geom.GeometryFactory;
+
+/**
+ * A base class for parsers that convert GeoJSON data into JTS Geometry objects.
+ * This class provides common functionality and a shared {@link GeometryFactory} for its subclasses.
+ * Subclasses of {@code BaseParser} use this shared geometry factory to construct specific types of geometries from
+ * their GeoJSON representations. By extending this base class, they inherit the common functionality and the ability
+ * to leverage the same geometry creation process.
+ */
+public class BaseParser {
+
+    protected GeometryFactory geometryFactory;
+
+    public BaseParser(GeometryFactory geometryFactory) {
+        this.geometryFactory = geometryFactory;
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/GenericGeometryParser.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/GenericGeometryParser.java
new file mode 100644
index 0000000..d1c9475
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/GenericGeometryParser.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * This file includes code derived from the project "jackson-datatype-jts"
+ * under the Apache License 2.0.
+ * Original source: https://github.com/bedatadriven/jackson-datatype-jts/blob/master/src/main/java/com/bedatadriven/jackson/datatype/jts/parsers/GenericGeometryParser.java.
+ *
+ * Modifications:
+ * - Adapted the code to support the org.locationtech.jts package instead of com.vividsolutions.jts
+ *
+ * The modified version retains the original license and notices. For more information
+ * on the original project and licensing, please visit https://github.com/bedatadriven/jackson-datatype-jts.
+ */
+package org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.parsers;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.GeoJsonConstants;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.JsonNode;
+
+/**
+ * A parser for converting various types of GeoJSON geometries into JTS {@link Geometry} objects using a specified {@link GeometryFactory}.
+ * This class acts as a general-purpose parser that can handle multiple types of GeoJSON geometries, including Point, MultiPoint,
+ * LineString, MultiLineString, Polygon, MultiPolygon, and GeometryCollection. It dynamically delegates the parsing to specific
+ * geometry parsers based on the GeoJSON type of the geometry.
+ * The parser extends {@code BaseParser} to utilize shared functionality and ensure consistent application of the {@link GeometryFactory}
+ * for creating JTS geometry instances. It maintains a registry of individual geometry parsers, each capable of handling a specific
+ * type of GeoJSON geometry.
+ */
+public class GenericGeometryParser extends BaseParser implements GeometryParser<Geometry> {
+
+    private Map<String, GeometryParser> parsers;
+
+    public GenericGeometryParser(GeometryFactory geometryFactory) {
+        super(geometryFactory);
+        parsers = new HashMap<>();
+        parsers.put(GeoJsonConstants.POINT, new PointParser(geometryFactory));
+        parsers.put(GeoJsonConstants.MULTI_POINT, new MultiPointParser(geometryFactory));
+        parsers.put(GeoJsonConstants.LINE_STRING, new LineStringParser(geometryFactory));
+        parsers.put(GeoJsonConstants.MULTI_LINE_STRING, new MultiLineStringParser(geometryFactory));
+        parsers.put(GeoJsonConstants.POLYGON, new PolygonParser(geometryFactory));
+        parsers.put(GeoJsonConstants.MULTI_POLYGON, new MultiPolygonParser(geometryFactory));
+        parsers.put(GeoJsonConstants.GEOMETRY_COLLECTION, new GeometryCollectionParser(geometryFactory, this));
+    }
+
+    @Override
+    public Geometry geometryFromJson(JsonNode node) throws JsonMappingException {
+        String typeName = node.get(GeoJsonConstants.TYPE).asText();
+        GeometryParser parser = parsers.get(typeName);
+        if (parser != null) {
+            return parser.geometryFromJson(node);
+        } else {
+            throw new JsonMappingException("Invalid geometry type: " + typeName);
+        }
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/GeometryCollectionParser.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/GeometryCollectionParser.java
new file mode 100644
index 0000000..a945675
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/GeometryCollectionParser.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * This file includes code derived from the project "jackson-datatype-jts"
+ * under the Apache License 2.0.
+ * Original source: https://github.com/bedatadriven/jackson-datatype-jts/blob/master/src/main/java/com/bedatadriven/jackson/datatype/jts/parsers/GeometryCollectionParser.java.
+ *
+ * Modifications:
+ * - Adapted the code to support the org.locationtech.jts package instead of com.vividsolutions.jts
+ *
+ * The modified version retains the original license and notices. For more information
+ * on the original project and licensing, please visit https://github.com/bedatadriven/jackson-datatype-jts.
+ */
+package org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.parsers;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.GeoJsonConstants;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryCollection;
+import org.locationtech.jts.geom.GeometryFactory;
+
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.JsonNode;
+
+/**
+ * A parser for converting GeoJSON geometry collection data into JTS {@link GeometryCollection} objects using a specified {@link GeometryFactory}.
+ * This class is designed to handle the parsing of GeoJSON representations of geometry collections, which may include multiple geometries
+ * of different types (e.g., Points, LineStrings, Polygons, etc.). It utilizes a {@link GenericGeometryParser} to handle the parsing of individual
+ * geometries within the collection.
+ *
+ * The parser extends {@code BaseParser} to leverage common functionality and ensure the consistent application of the {@link GeometryFactory}
+ * for creating {@link GeometryCollection} instances.
+ */
+public class GeometryCollectionParser extends BaseParser implements GeometryParser<GeometryCollection> {
+
+    private GenericGeometryParser genericGeometriesParser;
+
+    public GeometryCollectionParser(GeometryFactory geometryFactory, GenericGeometryParser genericGeometriesParser) {
+        super(geometryFactory);
+        this.genericGeometriesParser = genericGeometriesParser;
+    }
+
+    private Geometry[] geometriesFromJson(JsonNode arrayOfGeoms) throws JsonMappingException {
+        Geometry[] items = new Geometry[arrayOfGeoms.size()];
+        for (int i = 0; i != arrayOfGeoms.size(); ++i) {
+            items[i] = genericGeometriesParser.geometryFromJson(arrayOfGeoms.get(i));
+        }
+        return items;
+    }
+
+    @Override
+    public GeometryCollection geometryFromJson(JsonNode node) throws JsonMappingException {
+        return geometryFactory.createGeometryCollection(geometriesFromJson(node.get(GeoJsonConstants.GEOMETRIES)));
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/GeometryParser.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/GeometryParser.java
new file mode 100644
index 0000000..5e3bacc
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/GeometryParser.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * This file includes code derived from the project "jackson-datatype-jts"
+ * under the Apache License 2.0.
+ * Original source: https://github.com/bedatadriven/jackson-datatype-jts/blob/master/src/main/java/com/bedatadriven/jackson/datatype/jts/parsers/GeometryParser.java.
+ *
+ * Modifications:
+ * - Adapted the code to support the org.locationtech.jts package instead of com.vividsolutions.jts
+ *
+ * The modified version retains the original license and notices. For more information
+ * on the original project and licensing, please visit https://github.com/bedatadriven/jackson-datatype-jts.
+ */
+package org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.parsers;
+
+import org.locationtech.jts.geom.Geometry;
+
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.JsonNode;
+
+/**
+ * An interface for parsing GeoJSON data into JTS {@link Geometry} objects.
+ * This interface defines a contract for classes that convert JSON representations of geometries
+ * into specific JTS geometry instances, such as Points, LineStrings, Polygons, etc.
+ * The {@code GeometryParser} interface ensures a standard method is available for deserializing
+ * GeoJSON structures into their corresponding JTS geometrical forms. Implementations of this interface
+ * are responsible for handling the parsing logic for different types of geometries.
+ *
+ * @param <T> the type of JTS Geometry that the parser will produce, such as Point, LineString, Polygon, etc.
+ */
+public interface GeometryParser<T extends Geometry> {
+
+    T geometryFromJson(JsonNode node) throws JsonMappingException;
+
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/LineStringParser.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/LineStringParser.java
new file mode 100644
index 0000000..1c744db
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/LineStringParser.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * This file includes code derived from the project "jackson-datatype-jts"
+ * under the Apache License 2.0.
+ * Original source: https://github.com/bedatadriven/jackson-datatype-jts/blob/master/src/main/java/com/bedatadriven/jackson/datatype/jts/parsers/LineStringParser.java.
+ *
+ * Modifications:
+ * - Adapted the code to support the org.locationtech.jts package instead of com.vividsolutions.jts
+ *
+ * The modified version retains the original license and notices. For more information
+ * on the original project and licensing, please visit https://github.com/bedatadriven/jackson-datatype-jts.
+ */
+package org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.parsers;
+
+import static org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.GeoJsonConstants.COORDINATES;
+
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.LineString;
+
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.JsonNode;
+
+/**
+ * A parser for converting GeoJSON line string data into JTS {@link LineString} objects using a specified {@link GeometryFactory}.
+ * This class parses JSON representations of line strings, which are essentially ordered sets of coordinates that form a continuous line.
+ * It supports the extraction and transformation of coordinate arrays from GeoJSON into JTS LineString geometries.
+ * The parser extends {@code BaseParser} to utilize common functionalities and ensures the consistent application of the {@link GeometryFactory}
+ * for creating {@link LineString} instances.
+ */
+public class LineStringParser extends BaseParser implements GeometryParser<LineString> {
+
+    public LineStringParser(GeometryFactory geometryFactory) {
+        super(geometryFactory);
+    }
+
+    public LineString lineStringFromJson(JsonNode root) {
+        return geometryFactory.createLineString(PointParser.coordinatesFromJson(root.get(COORDINATES)));
+    }
+
+    @Override
+    public LineString geometryFromJson(JsonNode node) throws JsonMappingException {
+        return lineStringFromJson(node);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/MultiLineStringParser.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/MultiLineStringParser.java
new file mode 100644
index 0000000..9a7fbae
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/MultiLineStringParser.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * This file includes code derived from the project "jackson-datatype-jts"
+ * under the Apache License 2.0.
+ * Original source: https://github.com/bedatadriven/jackson-datatype-jts/blob/master/src/main/java/com/bedatadriven/jackson/datatype/jts/parsers/MultiLineStringParser.java.
+ *
+ * Modifications:
+ * - Adapted the code to support the org.locationtech.jts package instead of com.vividsolutions.jts
+ *
+ * The modified version retains the original license and notices. For more information
+ * on the original project and licensing, please visit https://github.com/bedatadriven/jackson-datatype-jts.
+ */
+package org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.parsers;
+
+import static org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.GeoJsonConstants.COORDINATES;
+
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.MultiLineString;
+
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.JsonNode;
+
+/**
+ * A parser for converting GeoJSON multi-line string data into JTS {@link MultiLineString} objects using a specified {@link GeometryFactory}.
+ * This class handles the parsing of multi-line string geometries, where each line string is represented by an array of coordinate arrays.
+ * It supports the conversion of complex multi-line string geometries, which may consist of multiple, distinct line strings.
+ * The parser extends {@code BaseParser} to utilize shared functionality and ensure the consistent application of the {@link GeometryFactory}
+ * in creating {@link MultiLineString} instances.
+ */
+public class MultiLineStringParser extends BaseParser implements GeometryParser<MultiLineString> {
+
+    public MultiLineStringParser(GeometryFactory geometryFactory) {
+        super(geometryFactory);
+    }
+
+    public MultiLineString multiLineStringFromJson(JsonNode root) {
+        return geometryFactory.createMultiLineString(lineStringsFromJson(root.get(COORDINATES)));
+    }
+
+    private LineString[] lineStringsFromJson(JsonNode array) {
+        LineString[] strings = new LineString[array.size()];
+        for (int i = 0; i != array.size(); ++i) {
+            strings[i] = geometryFactory.createLineString(PointParser.coordinatesFromJson(array.get(i)));
+        }
+        return strings;
+    }
+
+    @Override
+    public MultiLineString geometryFromJson(JsonNode node) throws JsonMappingException {
+        return multiLineStringFromJson(node);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/MultiPointParser.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/MultiPointParser.java
new file mode 100644
index 0000000..a27d60a
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/MultiPointParser.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * This file includes code derived from the project "jackson-datatype-jts"
+ * under the Apache License 2.0.
+ * Original source: https://github.com/bedatadriven/jackson-datatype-jts/blob/master/src/main/java/com/bedatadriven/jackson/datatype/jts/parsers/MultiPointParser.java.
+ *
+ * Modifications:
+ * - Adapted the code to support the org.locationtech.jts package instead of com.vividsolutions.jts
+ *
+ * The modified version retains the original license and notices. For more information
+ * on the original project and licensing, please visit https://github.com/bedatadriven/jackson-datatype-jts.
+ */
+package org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.parsers;
+
+import static org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.GeoJsonConstants.COORDINATES;
+
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.MultiPoint;
+
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.JsonNode;
+
+/**
+ * A parser for converting GeoJSON multi-point data into JTS {@link MultiPoint} objects using a specified {@link GeometryFactory}.
+ * This class is capable of parsing JSON representations of multi-point geometries, where each point is represented
+ * by an array of coordinates. The parser extends {@code BaseParser} to utilize common functionality and ensure
+ * the {@link GeometryFactory} is applied consistently to create {@link MultiPoint} instances.
+ */
+public class MultiPointParser extends BaseParser implements GeometryParser<MultiPoint> {
+
+    public MultiPointParser(GeometryFactory geometryFactory) {
+        super(geometryFactory);
+    }
+
+    public MultiPoint multiPointFromJson(JsonNode root) {
+        return geometryFactory.createMultiPointFromCoords(PointParser.coordinatesFromJson(root.get(COORDINATES)));
+    }
+
+    @Override
+    public MultiPoint geometryFromJson(JsonNode node) throws JsonMappingException {
+        return multiPointFromJson(node);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/MultiPolygonParser.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/MultiPolygonParser.java
new file mode 100644
index 0000000..c9a4235
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/MultiPolygonParser.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * This file includes code derived from the project "jackson-datatype-jts"
+ * under the Apache License 2.0.
+ * Original source: https://github.com/bedatadriven/jackson-datatype-jts/blob/master/src/main/java/com/bedatadriven/jackson/datatype/jts/parsers/MultiPolygonParser.java.
+ *
+ * Modifications:
+ * - Adapted the code to support the org.locationtech.jts package instead of com.vividsolutions.jts
+ *
+ * The modified version retains the original license and notices. For more information
+ * on the original project and licensing, please visit https://github.com/bedatadriven/jackson-datatype-jts.
+ */
+package org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.parsers;
+
+import static org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.GeoJsonConstants.COORDINATES;
+
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.MultiPolygon;
+import org.locationtech.jts.geom.Polygon;
+
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.JsonNode;
+
+/**
+ * A parser for converting GeoJSON multi-polygon data into JTS {@link MultiPolygon} objects using a specified {@link GeometryFactory}.
+ * This class leverages a helper {@link PolygonParser} to parse individual polygons from a collection of polygons represented
+ * in a GeoJSON format. It supports parsing complex multi-polygon geometries, which can include multiple outer boundaries and
+ * their respective inner holes.
+ * The parser extends {@code BaseParser} to make use of common functionalities and ensure the consistent application of the
+ * {@link GeometryFactory} in creating {@link MultiPolygon} instances.
+ */
+public class MultiPolygonParser extends BaseParser implements GeometryParser<MultiPolygon> {
+
+    private PolygonParser helperParser;
+
+    public MultiPolygonParser(GeometryFactory geometryFactory) {
+        super(geometryFactory);
+        helperParser = new PolygonParser(geometryFactory);
+    }
+
+    public MultiPolygon multiPolygonFromJson(JsonNode root) {
+        JsonNode arrayOfPolygons = root.get(COORDINATES);
+        return geometryFactory.createMultiPolygon(polygonsFromJson(arrayOfPolygons));
+    }
+
+    private Polygon[] polygonsFromJson(JsonNode arrayOfPolygons) {
+        Polygon[] polygons = new Polygon[arrayOfPolygons.size()];
+        for (int i = 0; i != arrayOfPolygons.size(); ++i) {
+            polygons[i] = helperParser.polygonFromJsonArrayOfRings(arrayOfPolygons.get(i));
+        }
+        return polygons;
+    }
+
+    @Override
+    public MultiPolygon geometryFromJson(JsonNode node) throws JsonMappingException {
+        return multiPolygonFromJson(node);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/PointParser.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/PointParser.java
new file mode 100644
index 0000000..a91a326
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/PointParser.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * This file includes code derived from the project "jackson-datatype-jts"
+ * under the Apache License 2.0.
+ * Original source: https://github.com/bedatadriven/jackson-datatype-jts/blob/master/src/main/java/com/bedatadriven/jackson/datatype/jts/parsers/PointParser.java.
+ *
+ * Modifications:
+ * - Adapted the code to support the org.locationtech.jts package instead of com.vividsolutions.jts
+ *
+ * The modified version retains the original license and notices. For more information
+ * on the original project and licensing, please visit https://github.com/bedatadriven/jackson-datatype-jts.
+ */
+package org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.parsers;
+
+import static org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.GeoJsonConstants.COORDINATES;
+
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.CoordinateXYZM;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.Point;
+
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.JsonNode;
+
+/**
+ * Parses JSON representations of GeoJSON Points into JTS {@link Point} objects.
+ * This parser handles the extraction of coordinates from GeoJSON and converts them
+ * into {@link Point} geometries using a provided {@link GeometryFactory}.
+ *
+ * This class supports reading points defined with two-dimensional (x, y),
+ * three-dimensional (x, y, z), and four-dimensional (x, y, z, m) coordinates.
+ */
+public class PointParser extends BaseParser implements GeometryParser<Point> {
+
+    public PointParser(GeometryFactory geometryFactory) {
+        super(geometryFactory);
+    }
+
+    public static Coordinate coordinateFromJson(JsonNode array) {
+        assert array.isArray() && (array.size() == 2 || array.size() == 3
+                || array.size() == 4) : "expecting coordinate array with single point [ x, y, |z|, |m| ]";
+
+        if (array.size() == 2) {
+            return new Coordinate(array.get(0).asDouble(), array.get(1).asDouble());
+        }
+
+        if (array.size() == 3) {
+            return new Coordinate(array.get(0).asDouble(), array.get(1).asDouble(), array.get(2).asDouble());
+        }
+
+        return new CoordinateXYZM(array.get(0).asDouble(), array.get(1).asDouble(), array.get(2).asDouble(),
+                array.get(3).asDouble());
+    }
+
+    public static Coordinate[] coordinatesFromJson(JsonNode array) {
+        Coordinate[] points = new Coordinate[array.size()];
+        for (int i = 0; i != array.size(); ++i) {
+            points[i] = PointParser.coordinateFromJson(array.get(i));
+        }
+        return points;
+    }
+
+    public Point pointFromJson(JsonNode node) {
+        return geometryFactory.createPoint(coordinateFromJson(node.get(COORDINATES)));
+    }
+
+    @Override
+    public Point geometryFromJson(JsonNode node) throws JsonMappingException {
+        return pointFromJson(node);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/PolygonParser.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/PolygonParser.java
new file mode 100644
index 0000000..5c7492a
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/PolygonParser.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * This file includes code derived from the project "jackson-datatype-jts"
+ * under the Apache License 2.0.
+ * Original source: https://github.com/bedatadriven/jackson-datatype-jts/blob/master/src/main/java/com/bedatadriven/jackson/datatype/jts/parsers/PolygonParser.java.
+ *
+ * Modifications:
+ * - Adapted the code to support the org.locationtech.jts package instead of com.vividsolutions.jts
+ *
+ * The modified version retains the original license and notices. For more information
+ * on the original project and licensing, please visit https://github.com/bedatadriven/jackson-datatype-jts.
+ */
+package org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.parsers;
+
+import static org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.GeoJsonConstants.COORDINATES;
+
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.LinearRing;
+import org.locationtech.jts.geom.Polygon;
+
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.JsonNode;
+
+/**
+ * A parser for transforming GeoJSON polygon data into JTS {@link Polygon} objects using a specified {@link GeometryFactory}.
+ * This class handles the parsing of polygons, which may include an outer boundary and any number of inner holes.
+ * Each polygon is defined by arrays of coordinates that represent linear rings—the first array defines the exterior boundary,
+ * and any subsequent arrays define interior holes.
+ * This parser extends {@code BaseParser} to leverage shared functionality and ensure consistent application of the
+ * {@link GeometryFactory} in creating polygon geometries.
+ */
+public class PolygonParser extends BaseParser implements GeometryParser<Polygon> {
+
+    public PolygonParser(GeometryFactory geometryFactory) {
+        super(geometryFactory);
+    }
+
+    public Polygon polygonFromJson(JsonNode node) {
+        JsonNode arrayOfRings = node.get(COORDINATES);
+        return polygonFromJsonArrayOfRings(arrayOfRings);
+    }
+
+    public Polygon polygonFromJsonArrayOfRings(JsonNode arrayOfRings) {
+        LinearRing shell = linearRingsFromJson(arrayOfRings.get(0));
+        int size = arrayOfRings.size();
+        LinearRing[] holes = new LinearRing[size - 1];
+        for (int i = 1; i < size; i++) {
+            holes[i - 1] = linearRingsFromJson(arrayOfRings.get(i));
+        }
+        return geometryFactory.createPolygon(shell, holes);
+    }
+
+    private LinearRing linearRingsFromJson(JsonNode coordinates) {
+        assert coordinates.isArray() : "expected coordinates array";
+        return geometryFactory.createLinearRing(PointParser.coordinatesFromJson(coordinates));
+    }
+
+    @Override
+    public Polygon geometryFromJson(JsonNode node) throws JsonMappingException {
+        return polygonFromJson(node);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/serde/GeometryDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/serde/GeometryDeserializer.java
new file mode 100644
index 0000000..1cebfcd
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/serde/GeometryDeserializer.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * This file includes code derived from the project "jackson-datatype-jts"
+ * under the Apache License 2.0.
+ * Original source: https://github.com/bedatadriven/jackson-datatype-jts/blob/master/src/main/java/com/bedatadriven/jackson/datatype/jts/serialization/GeometryDeserializer.java.
+ *
+ * Modifications:
+ * - Adapted the code to support the org.locationtech.jts package instead of com.vividsolutions.jts
+ *
+ * The modified version retains the original license and notices. For more information
+ * on the original project and licensing, please visit https://github.com/bedatadriven/jackson-datatype-jts.
+ */
+package org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.serde;
+
+import java.io.IOException;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.parsers.GeometryParser;
+import org.locationtech.jts.geom.Geometry;
+
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.ObjectCodec;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.JsonDeserializer;
+import com.fasterxml.jackson.databind.JsonNode;
+
+/**
+ * A custom Jackson deserializer for JTS Geometry objects.
+ * This deserializer translates JSON structures into JTS Geometry instances using a specified
+ * {@link GeometryParser}. It supports generic geometry types, allowing for flexible deserialization
+ * of various specific types of geometries such as Point, LineString, Polygon, etc.
+ * The deserializer relies on a geometry parser which must be provided during instantiation.
+ * The parser is responsible for converting a JSON node into a corresponding JTS Geometry object.
+ * Usage:
+ * This deserializer is registered in the Jackson JTS
+ * module {@link org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.JtsModule} and is used to configure an
+ * ObjectMapper to support JTS geometries.
+ */
+public class GeometryDeserializer<T extends Geometry> extends JsonDeserializer<T> {
+
+    private GeometryParser<T> geometryParser;
+
+    public GeometryDeserializer(GeometryParser<T> geometryParser) {
+        this.geometryParser = geometryParser;
+    }
+
+    /**
+     * Deserializes a JSON node into a JTS Geometry object.
+     * The JSON node is processed by the configured GeometryParser to produce the Geometry instance.
+     *
+     * @param jsonParser the Jackson parser reading the JSON content
+     * @param deserializationContext the Jackson deserialization context
+     * @return the deserialized JTS Geometry object
+     * @throws IOException if there is an issue in reading or parsing the JSON node
+     */
+    @Override
+    public T deserialize(JsonParser jsonParser, DeserializationContext deserializationContext) throws IOException {
+        ObjectCodec oc = jsonParser.getCodec();
+        JsonNode root = oc.readTree(jsonParser);
+        return geometryParser.geometryFromJson(root);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/serde/GeometrySerializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/serde/GeometrySerializer.java
new file mode 100644
index 0000000..3207afb
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/serde/GeometrySerializer.java
@@ -0,0 +1,231 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * This file includes code derived from the project "jackson-datatype-jts"
+ * under the Apache License 2.0.
+ * Original source: https://github.com/bedatadriven/jackson-datatype-jts/blob/master/src/main/java/com/bedatadriven/jackson/datatype/jts/serialization/GeometrySerializer.java.
+ *
+ * Modifications:
+ * - Adapted the code to support the org.locationtech.jts package instead of com.vividsolutions.jts
+ *
+ * The modified version retains the original license and notices. For more information
+ * on the original project and licensing, please visit https://github.com/bedatadriven/jackson-datatype-jts.
+ */
+package org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.serde;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.GeoJsonConstants;
+import org.locationtech.jts.geom.CoordinateXYZM;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryCollection;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.MultiLineString;
+import org.locationtech.jts.geom.MultiPoint;
+import org.locationtech.jts.geom.MultiPolygon;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.JsonSerializer;
+import com.fasterxml.jackson.databind.SerializerProvider;
+
+/**
+ * A custom Jackson serializer for JTS Geometry objects that translates these objects into their GeoJSON representations.
+ * This class supports serialization for all primary JTS geometry types including Point, LineString, Polygon, and their
+ * respective collections such as MultiPoint, MultiLineString, MultiPolygon, and GeometryCollection.
+ * It handles complex geometries by delegating to specific methods based on the instance type of the geometry object,
+ * ensuring that each geometry type is correctly represented according to the GeoJSON standard.
+ */
+public class GeometrySerializer extends JsonSerializer<Geometry> {
+
+    @Override
+    public void serialize(Geometry value, JsonGenerator jsonGenerator, SerializerProvider provider) throws IOException {
+        writeGeometry(jsonGenerator, value);
+    }
+
+    /**
+     * Writes the geometry object to the JsonGenerator. This method determines the type of the geometry
+     * and calls the appropriate method to handle the serialization.
+     *
+     * @param jsonGenerator the JsonGenerator to use for writing the GeoJSON
+     * @param value the Geometry object to serialize
+     * @throws IOException if an input/output error occurs
+     */
+    public void writeGeometry(JsonGenerator jsonGenerator, Geometry value) throws IOException {
+        if (value instanceof Polygon) {
+            writePolygon(jsonGenerator, (Polygon) value);
+
+        } else if (value instanceof Point) {
+            writePoint(jsonGenerator, (Point) value);
+
+        } else if (value instanceof MultiPoint) {
+            writeMultiPoint(jsonGenerator, (MultiPoint) value);
+
+        } else if (value instanceof MultiPolygon) {
+            writeMultiPolygon(jsonGenerator, (MultiPolygon) value);
+
+        } else if (value instanceof LineString) {
+            writeLineString(jsonGenerator, (LineString) value);
+
+        } else if (value instanceof MultiLineString) {
+            writeMultiLineString(jsonGenerator, (MultiLineString) value);
+
+        } else if (value instanceof GeometryCollection) {
+            writeGeometryCollection(jsonGenerator, (GeometryCollection) value);
+
+        } else {
+            throw new JsonMappingException(jsonGenerator,
+                    "Geometry type " + value.getClass().getName() + " cannot be serialized as GeoJSON."
+                            + "Supported types are: "
+                            + Arrays.asList(Point.class.getName(), LineString.class.getName(), Polygon.class.getName(),
+                                    MultiPoint.class.getName(), MultiLineString.class.getName(),
+                                    MultiPolygon.class.getName(), GeometryCollection.class.getName()));
+        }
+    }
+
+    private void writeGeometryCollection(JsonGenerator jsonGenerator, GeometryCollection value) throws IOException {
+        jsonGenerator.writeStartObject();
+        jsonGenerator.writeStringField(GeoJsonConstants.TYPE, GeoJsonConstants.GEOMETRY_COLLECTION);
+        jsonGenerator.writeArrayFieldStart(GeoJsonConstants.GEOMETRIES);
+
+        for (int i = 0; i != value.getNumGeometries(); ++i) {
+            writeGeometry(jsonGenerator, value.getGeometryN(i));
+        }
+
+        jsonGenerator.writeEndArray();
+        jsonGenerator.writeEndObject();
+    }
+
+    private void writeMultiPoint(JsonGenerator jsonGenerator, MultiPoint value) throws IOException {
+        jsonGenerator.writeStartObject();
+        jsonGenerator.writeStringField(GeoJsonConstants.TYPE, GeoJsonConstants.MULTI_POINT);
+        jsonGenerator.writeArrayFieldStart(GeoJsonConstants.COORDINATES);
+
+        for (int i = 0; i != value.getNumGeometries(); ++i) {
+            writePointCoordinates(jsonGenerator, (Point) value.getGeometryN(i));
+        }
+
+        jsonGenerator.writeEndArray();
+        jsonGenerator.writeEndObject();
+    }
+
+    private void writeMultiLineString(JsonGenerator jsonGenerator, MultiLineString value) throws IOException {
+        jsonGenerator.writeStartObject();
+        jsonGenerator.writeStringField(GeoJsonConstants.TYPE, GeoJsonConstants.MULTI_LINE_STRING);
+        jsonGenerator.writeArrayFieldStart(GeoJsonConstants.COORDINATES);
+
+        for (int i = 0; i != value.getNumGeometries(); ++i) {
+            writeLineStringCoordinates(jsonGenerator, (LineString) value.getGeometryN(i));
+        }
+
+        jsonGenerator.writeEndArray();
+        jsonGenerator.writeEndObject();
+    }
+
+    @Override
+    public Class<Geometry> handledType() {
+        return Geometry.class;
+    }
+
+    private void writeMultiPolygon(JsonGenerator jsonGenerator, MultiPolygon value) throws IOException {
+        jsonGenerator.writeStartObject();
+        jsonGenerator.writeStringField(GeoJsonConstants.TYPE, GeoJsonConstants.MULTI_POLYGON);
+        jsonGenerator.writeArrayFieldStart(GeoJsonConstants.COORDINATES);
+
+        for (int i = 0; i != value.getNumGeometries(); ++i) {
+            writePolygonCoordinates(jsonGenerator, (Polygon) value.getGeometryN(i));
+        }
+
+        jsonGenerator.writeEndArray();
+        jsonGenerator.writeEndObject();
+    }
+
+    private void writePolygon(JsonGenerator jsonGenerator, Polygon value) throws IOException {
+        jsonGenerator.writeStartObject();
+        jsonGenerator.writeStringField(GeoJsonConstants.TYPE, GeoJsonConstants.POLYGON);
+        jsonGenerator.writeFieldName(GeoJsonConstants.COORDINATES);
+        writePolygonCoordinates(jsonGenerator, value);
+
+        jsonGenerator.writeEndObject();
+    }
+
+    private void writePolygonCoordinates(JsonGenerator jsonGenerator, Polygon value) throws IOException {
+        jsonGenerator.writeStartArray();
+        writeLineStringCoordinates(jsonGenerator, value.getExteriorRing());
+
+        for (int i = 0; i < value.getNumInteriorRing(); ++i) {
+            writeLineStringCoordinates(jsonGenerator, value.getInteriorRingN(i));
+        }
+        jsonGenerator.writeEndArray();
+    }
+
+    private void writeLineStringCoordinates(JsonGenerator jsonGenerator, LineString ring) throws IOException {
+        jsonGenerator.writeStartArray();
+        for (int i = 0; i != ring.getNumPoints(); ++i) {
+            Point p = ring.getPointN(i);
+            writePointCoordinates(jsonGenerator, p);
+        }
+        jsonGenerator.writeEndArray();
+    }
+
+    private void writeLineString(JsonGenerator jsonGenerator, LineString lineString) throws IOException {
+        jsonGenerator.writeStartObject();
+        jsonGenerator.writeStringField(GeoJsonConstants.TYPE, GeoJsonConstants.LINE_STRING);
+        jsonGenerator.writeFieldName(GeoJsonConstants.COORDINATES);
+        writeLineStringCoordinates(jsonGenerator, lineString);
+        jsonGenerator.writeEndObject();
+    }
+
+    private void writePoint(JsonGenerator jsonGenerator, Point p) throws IOException {
+        jsonGenerator.writeStartObject();
+        jsonGenerator.writeStringField(GeoJsonConstants.TYPE, GeoJsonConstants.POINT);
+        jsonGenerator.writeFieldName(GeoJsonConstants.COORDINATES);
+        writePointCoordinates(jsonGenerator, p);
+        jsonGenerator.writeEndObject();
+    }
+
+    private void writePointCoordinates(JsonGenerator jsonGenerator, Point p) throws IOException {
+        jsonGenerator.writeStartArray();
+
+        writeFormattedNumber(jsonGenerator, p.getCoordinate().x);
+        writeFormattedNumber(jsonGenerator, p.getCoordinate().y);
+
+        if (!Double.isNaN(p.getCoordinate().z)) {
+            writeFormattedNumber(jsonGenerator, p.getCoordinate().z);
+        }
+
+        if (p.getCoordinate() instanceof CoordinateXYZM) {
+            double m = p.getCoordinate().getM();
+            writeFormattedNumber(jsonGenerator, m);
+        }
+        jsonGenerator.writeEndArray();
+    }
+
+    private void writeFormattedNumber(JsonGenerator jsonGenerator, double value) throws IOException {
+        if ((value == Math.floor(value)) && !Double.isInfinite(value)) {
+            jsonGenerator.writeNumber((int) value);
+        } else {
+            jsonGenerator.writeNumber(value);
+        }
+    }
+
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/CSVPrinterFactoryProvider.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/CSVPrinterFactoryProvider.java
index b8201ae..322b3e6 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/CSVPrinterFactoryProvider.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/CSVPrinterFactoryProvider.java
@@ -18,6 +18,15 @@
  */
 package org.apache.asterix.formats.nontagged;
 
+import static org.apache.asterix.dataflow.data.nontagged.printers.csv.CSVUtils.KEY_DELIMITER;
+import static org.apache.asterix.dataflow.data.nontagged.printers.csv.CSVUtils.KEY_ESCAPE;
+import static org.apache.asterix.dataflow.data.nontagged.printers.csv.CSVUtils.KEY_FORCE_QUOTE;
+import static org.apache.asterix.dataflow.data.nontagged.printers.csv.CSVUtils.KEY_NULL;
+import static org.apache.asterix.dataflow.data.nontagged.printers.csv.CSVUtils.KEY_QUOTE;
+
+import java.util.Collections;
+import java.util.Map;
+
 import org.apache.asterix.dataflow.data.nontagged.printers.adm.ShortWithoutTypeInfoPrinterFactory;
 import org.apache.asterix.dataflow.data.nontagged.printers.csv.ABooleanPrinterFactory;
 import org.apache.asterix.dataflow.data.nontagged.printers.csv.ACirclePrinterFactory;
@@ -52,12 +61,26 @@
 import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
 import org.apache.hyracks.algebricks.data.IPrinterFactoryProvider;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class CSVPrinterFactoryProvider implements IPrinterFactoryProvider {
+    private ARecordType itemType;
+    private Map<String, String> configuration;
+    private SourceLocation sourceLocation;
 
-    public static final CSVPrinterFactoryProvider INSTANCE = new CSVPrinterFactoryProvider();
+    public static final CSVPrinterFactoryProvider INSTANCE =
+            new CSVPrinterFactoryProvider(null, Collections.emptyMap(), null);
 
-    private CSVPrinterFactoryProvider() {
+    public static final CSVPrinterFactoryProvider createInstance(ARecordType itemType,
+            Map<String, String> configuration, SourceLocation sourceLocation) {
+        return new CSVPrinterFactoryProvider(itemType, configuration, sourceLocation);
+    }
+
+    private CSVPrinterFactoryProvider(ARecordType itemType, Map<String, String> configuration,
+            SourceLocation sourceLocation) {
+        this.itemType = itemType;
+        this.configuration = configuration;
+        this.sourceLocation = sourceLocation;
     }
 
     @Override
@@ -76,7 +99,7 @@
                     return AInt64PrinterFactory.INSTANCE;
                 case MISSING:
                 case NULL:
-                    return ANullPrinterFactory.INSTANCE;
+                    ANullPrinterFactory.createInstance(configuration.get(KEY_NULL));
                 case BOOLEAN:
                     return ABooleanPrinterFactory.INSTANCE;
                 case FLOAT:
@@ -110,13 +133,15 @@
                 case RECTANGLE:
                     return ARectanglePrinterFactory.INSTANCE;
                 case STRING:
-                    return AStringPrinterFactory.INSTANCE;
+                    return AStringPrinterFactory.createInstance(configuration.get(KEY_QUOTE),
+                            configuration.get(KEY_FORCE_QUOTE), configuration.get(KEY_ESCAPE),
+                            configuration.get(KEY_DELIMITER));
                 case OBJECT:
-                    return new ARecordPrinterFactory((ARecordType) type);
+                    return new ARecordPrinterFactory((ARecordType) type, itemType, configuration);
                 case ARRAY:
-                    throw new NotImplementedException("'Orderedlist' type unsupported for CSV output");
+                    throw new NotImplementedException("'OrderedList' type unsupported for CSV output");
                 case MULTISET:
-                    throw new NotImplementedException("'Unorderedlist' type unsupported for CSV output");
+                    throw new NotImplementedException("'UnorderedList' type unsupported for CSV output");
                 case UNION:
                     if (((AUnionType) type).isUnknownableType()) {
                         return new AOptionalFieldPrinterFactory((AUnionType) type);
@@ -142,7 +167,7 @@
                     break;
             }
         }
-        return AObjectPrinterFactory.INSTANCE;
+        return AObjectPrinterFactory.createInstance(itemType, configuration);
 
     }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AGeometry.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AGeometry.java
index 84a2e81..c9d66b4 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AGeometry.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AGeometry.java
@@ -20,22 +20,23 @@
 
 import java.io.IOException;
 
+import org.apache.asterix.dataflow.data.nontagged.serde.jacksonjts.JtsModule;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
+import org.locationtech.jts.geom.Geometry;
 
-import com.esri.core.geometry.ogc.OGCGeometry;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.node.ObjectNode;
 
 public class AGeometry implements IAObject {
 
-    protected OGCGeometry geometry;
+    protected Geometry geometry;
 
-    public AGeometry(OGCGeometry geometry) {
+    public AGeometry(Geometry geometry) {
         this.geometry = geometry;
     }
 
-    public OGCGeometry getGeometry() {
+    public Geometry getGeometry() {
         return geometry;
     }
 
@@ -67,11 +68,13 @@
     @Override
     public ObjectNode toJSON() {
         ObjectMapper om = new ObjectMapper();
-        ObjectNode json = null;
+        om.registerModule(new JtsModule());
+        ObjectNode json;
         try {
-            json = (ObjectNode) om.readTree(geometry.asGeoJson());
+            String geoJson = om.writeValueAsString(geometry);
+            json = (ObjectNode) om.readTree(geoJson);
         } catch (IOException e) {
-            throw new RuntimeException(e);
+            return om.createObjectNode();
         }
         return json;
     }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AMutableGeometry.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AMutableGeometry.java
index 346d68a..925a0c9 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AMutableGeometry.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AMutableGeometry.java
@@ -18,29 +18,32 @@
  */
 package org.apache.asterix.om.base;
 
-import com.esri.core.geometry.OGCStructure;
-import com.esri.core.geometry.OperatorImportFromWkt;
-import com.esri.core.geometry.SpatialReference;
-import com.esri.core.geometry.WktImportFlags;
-import com.esri.core.geometry.ogc.OGCGeometry;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.io.ParseException;
+import org.locationtech.jts.io.WKTReader;
 
 public class AMutableGeometry extends AGeometry {
+    private Geometry geometry;
+    private final WKTReader wktReader = new WKTReader();
 
-    private OperatorImportFromWkt wktImporter;
-
-    public AMutableGeometry(OGCGeometry geom) {
+    public AMutableGeometry(Geometry geom) {
         super(geom);
-        wktImporter = OperatorImportFromWkt.local();
-    }
-
-    public void setValue(OGCGeometry geom) {
         this.geometry = geom;
     }
 
-    public void parseWKT(String wkt) {
-        OGCStructure structure;
+    public void setValue(Geometry geom) {
+        this.geometry = geom;
+    }
 
-        structure = wktImporter.executeOGC(WktImportFlags.wktImportNonTrusted, wkt, null);
-        this.geometry = OGCGeometry.createFromOGCStructure(structure, SpatialReference.create(4326));
+    public Geometry getGeometry() {
+        return this.geometry;
+    }
+
+    public void parseWKT(String wkt) {
+        try {
+            this.geometry = wktReader.read(wkt);
+        } catch (ParseException e) {
+            throw new RuntimeException(e);
+        }
     }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/dictionary/AbstractFieldNamesDictionary.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/dictionary/AbstractFieldNamesDictionary.java
new file mode 100644
index 0000000..b2cd223
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/dictionary/AbstractFieldNamesDictionary.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.dictionary;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
+import org.apache.asterix.om.base.AMutableString;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.util.string.UTF8StringReader;
+import org.apache.hyracks.util.string.UTF8StringWriter;
+
+public abstract class AbstractFieldNamesDictionary implements IFieldNamesDictionary {
+    /**
+     * Dummy field name used to add a column when encountering empty object
+     */
+    public static final IValueReference DUMMY_FIELD_NAME;
+    public static final int DUMMY_FIELD_NAME_INDEX = -1;
+
+    //For declared fields
+    private final AMutableString mutableString;
+    private final AStringSerializerDeserializer stringSerDer;
+
+    static {
+        VoidPointable dummy = new VoidPointable();
+        dummy.set(new byte[0], 0, 0);
+        DUMMY_FIELD_NAME = dummy;
+    }
+
+    AbstractFieldNamesDictionary() {
+        mutableString = new AMutableString("");
+        stringSerDer = new AStringSerializerDeserializer(new UTF8StringWriter(), new UTF8StringReader());
+    }
+
+    public static IFieldNamesDictionary create() {
+        return new FieldNamesTrieDictionary();
+    }
+
+    public static IFieldNamesDictionary deserialize(DataInput input) throws IOException {
+        return FieldNamesTrieDictionary.deserialize(input);
+    }
+
+    static ArrayBackedValueStorage creatFieldName(IValueReference fieldName) throws HyracksDataException {
+        ArrayBackedValueStorage copy = new ArrayBackedValueStorage(fieldName.getLength());
+        copy.append(fieldName);
+        return copy;
+    }
+
+    protected ArrayBackedValueStorage creatFieldName(String fieldName) throws HyracksDataException {
+        ArrayBackedValueStorage serializedFieldName = new ArrayBackedValueStorage();
+        serializeFieldName(fieldName, serializedFieldName);
+        return serializedFieldName;
+    }
+
+    protected void serializeFieldName(String fieldName, ArrayBackedValueStorage storage) throws HyracksDataException {
+        mutableString.setValue(fieldName);
+        stringSerDer.serialize(mutableString, storage.getDataOutput());
+    }
+
+    static void deserializeFieldNames(DataInput input, List<IValueReference> fieldNames, int numberOfFieldNames)
+            throws IOException {
+        for (int i = 0; i < numberOfFieldNames; i++) {
+            int length = input.readInt();
+            ArrayBackedValueStorage fieldName = new ArrayBackedValueStorage(length);
+            fieldName.setSize(length);
+            input.readFully(fieldName.getByteArray(), 0, length);
+            fieldNames.add(fieldName);
+        }
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/dictionary/ByteToNodeMap.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/dictionary/ByteToNodeMap.java
new file mode 100644
index 0000000..7bfae0b
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/dictionary/ByteToNodeMap.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.dictionary;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collection;
+
+import it.unimi.dsi.fastutil.objects.ObjectArrays;
+
+final class ByteToNodeMap {
+    private static final TrieNode[] EMPTY = new TrieNode[0];
+    private TrieNode[] children;
+    private int numberOfChildren;
+
+    ByteToNodeMap() {
+        children = EMPTY;
+        numberOfChildren = 0;
+    }
+
+    private ByteToNodeMap(TrieNode[] children, int numberOfChildren) {
+        this.children = children;
+        this.numberOfChildren = numberOfChildren;
+    }
+
+    void put(byte key, TrieNode node) {
+        int index = Byte.toUnsignedInt(key);
+        ensure(index);
+        children[index] = node;
+        numberOfChildren++;
+    }
+
+    TrieNode get(byte key) {
+        int index = Byte.toUnsignedInt(key);
+        if (index < children.length) {
+            return children[index];
+        }
+
+        return null;
+    }
+
+    private void ensure(int index) {
+        if (index >= children.length) {
+            children = ObjectArrays.grow(children, index + 1, children.length);
+        }
+    }
+
+    void addAllChildren(Collection<TrieNode> collection) {
+        int addedChildren = 0;
+        for (int i = 0; i < children.length && addedChildren < numberOfChildren; i++) {
+            TrieNode child = children[i];
+            if (child != null) {
+                collection.add(children[i]);
+                addedChildren++;
+            }
+        }
+    }
+
+    void serialize(DataOutput out) throws IOException {
+        out.writeInt(numberOfChildren);
+        out.writeInt(children.length);
+        int addedChildren = 0;
+        for (int i = 0; i < children.length && addedChildren < numberOfChildren; i++) {
+            TrieNode child = children[i];
+            if (child != null) {
+                out.writeInt(i);
+                child.serialize(out);
+                addedChildren++;
+            }
+        }
+    }
+
+    static ByteToNodeMap deserialize(DataInput in) throws IOException {
+        int numberOfChildren = in.readInt();
+        int length = in.readInt();
+        TrieNode[] children = length == 0 ? EMPTY : new TrieNode[length];
+        for (int i = 0; i < numberOfChildren; i++) {
+            int index = in.readInt();
+            children[index] = TrieNode.deserialize(in);
+        }
+
+        return new ByteToNodeMap(children, numberOfChildren);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/dictionary/FieldNameTrie.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/dictionary/FieldNameTrie.java
new file mode 100644
index 0000000..5333af5
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/dictionary/FieldNameTrie.java
@@ -0,0 +1,296 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.dictionary;
+
+import static org.apache.asterix.om.dictionary.AbstractFieldNamesDictionary.deserializeFieldNames;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Queue;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.util.string.UTF8StringUtil;
+
+public class FieldNameTrie {
+    private static final int VERSION = 1;
+    private final LookupState lookupState;
+
+    private final List<IValueReference> fieldNames;
+    private TrieNode rootNode;
+
+    public FieldNameTrie() {
+        this(new ArrayList<>());
+    }
+
+    private FieldNameTrie(List<IValueReference> fieldNames) {
+        this.fieldNames = fieldNames;
+        this.rootNode = new TrieNode();
+        lookupState = new LookupState();
+    }
+
+    public int insert(IValueReference fieldName) throws HyracksDataException {
+        int presentIndex = lookup(fieldName);
+        if (presentIndex == TrieNode.NOT_FOUND_INDEX) {
+            presentIndex = hookup(FieldNamesTrieDictionary.creatFieldName(fieldName));
+        }
+        return presentIndex;
+    }
+
+    public int lookup(IValueReference fieldName) {
+        //noinspection DuplicatedCode
+        int len = UTF8StringUtil.getUTFLength(fieldName.getByteArray(), fieldName.getStartOffset());
+        int start = fieldName.getStartOffset() + UTF8StringUtil.getNumBytesToStoreLength(len);
+        byte[] bytes = fieldName.getByteArray();
+
+        TrieNode searchNode = rootNode;
+        TrieNode prevNode = searchNode;
+
+        int byteIndex = start;
+        // previousByteIndex should point to the first byte to be compared
+        // when inserting the fieldName
+        int previousByteIndex = byteIndex;
+
+        int lastIndex = (start + len - 1);
+        while (byteIndex <= lastIndex) {
+            byte b = bytes[byteIndex];
+
+            TrieNode nextNode = searchNode.getChild(b);
+            if (nextNode == null) {
+                // saving state in case hookup is requested
+                lookupState.setState(prevNode, start, previousByteIndex, len);
+                return TrieNode.NOT_FOUND_INDEX;
+            }
+            // if the node exists, then compare the remaining byte seq.
+            prevNode = searchNode;
+            searchNode = nextNode;
+
+            if (searchNode.getLength() > 1) { // first byte will be same as byteIndex
+                // compare the stored sequence.
+                int fieldBytesLeftToCompare = lastIndex - byteIndex + 1;
+                // if the stored sequence in node is greater than the input field's
+                // byte to compare, then the result won't be there.
+                if (fieldBytesLeftToCompare < searchNode.getLength()) {
+                    // saving state in case hookup is requested
+                    lookupState.setState(prevNode, start, byteIndex, len);
+                    return TrieNode.NOT_FOUND_INDEX;
+                }
+
+                int c = 0;
+                byte[] storedFieldBytes = fieldNames.get(searchNode.getIndex()).getByteArray();
+                int storedFieldStart = searchNode.getStart();
+                previousByteIndex = byteIndex;
+                while (c < searchNode.getLength()) {
+                    if (bytes[byteIndex] != storedFieldBytes[storedFieldStart + c]) {
+                        // saving state in case hookup is requested
+                        // will restart from oldByteIndex, to make logic simpler.
+                        // other way could have been to store the splitIndex.
+                        lookupState.setState(prevNode, start, previousByteIndex, len);
+                        return TrieNode.NOT_FOUND_INDEX;
+                    }
+                    c++;
+                    byteIndex++;
+                }
+            } else {
+                previousByteIndex = byteIndex;
+                byteIndex++;
+            }
+        }
+
+        // saving state in case hookup is requested
+        lookupState.setState(prevNode, start, previousByteIndex, len);
+        return searchNode.isEndOfField() ? searchNode.getIndex() : TrieNode.NOT_FOUND_INDEX;
+    }
+
+    private int hookup(IValueReference fieldName) {
+        // since lookup operation always precedes a hookup operation
+        // we can use the saved state to start hookup.
+        int len = lookupState.getFieldLength();
+        TrieNode searchNode = lookupState.getLastNode();
+
+        // resume from the stored node.
+        int bytesToStoreLength = UTF8StringUtil.getNumBytesToStoreLength(len);
+
+        int byteIndex = lookupState.getRelativeOffsetFromStart() + bytesToStoreLength;
+        byte[] bytes = fieldName.getByteArray();
+        int lastIndex = (bytesToStoreLength + len - 1);
+        while (byteIndex <= lastIndex) {
+            byte b = bytes[byteIndex];
+            TrieNode nextNode = searchNode.getChild(b);
+            if (nextNode == null) {
+                // since there no such node, then create a node, and put
+                // rest bytes in the nodes.
+                TrieNode childNode = new TrieNode();
+                // first insert, then add the field
+                // start from byteIndex with newLength.
+                // newLength = lastIndex - byteIndex + 1
+                childNode.setIndex(fieldNames.size(), byteIndex, lastIndex - byteIndex + 1, bytesToStoreLength);
+                childNode.setIsEndOfField(true);
+                fieldNames.add(fieldName);
+
+                searchNode.putChild(b, childNode);
+                return childNode.getIndex();
+            }
+            // if node exists, compare the remaining byte seq
+            searchNode = nextNode;
+
+            if (searchNode.getLength() > 1) {
+                // compare the byte seq
+                int c = 0;
+                int oldByteIndex = byteIndex;
+
+                IValueReference storedFieldName = fieldNames.get(searchNode.getIndex());
+                byte[] storedFieldBytes = storedFieldName.getByteArray();
+                int storedFieldStart = searchNode.getStart();
+                while (c < Math.min(searchNode.getLength(), lastIndex - oldByteIndex + 1)) {
+                    if (bytes[byteIndex] != storedFieldBytes[storedFieldStart + c]) {
+                        break;
+                    }
+                    c++;
+                    byteIndex++;
+                }
+
+                // from c & byteIndex, things are not matching,
+                // split into two nodes,
+                // one from (c, ...) -> handled below
+                // other from (byteIndex, ...) -> handled in the next iteration, as byteIndex will be absent.
+
+                // handling (c, ..)
+                int leftToSplitForCurrentNode = searchNode.getLength() - c;
+                if (leftToSplitForCurrentNode > 0) {
+                    searchNode.split(storedFieldName, c);
+                }
+            } else {
+                byteIndex++;
+            }
+        }
+
+        // since the node is already present,
+        // point it to the current fieldName, and update the start and length based on the fieldName
+        // prefix would be the same
+        // find absolute starting point in the current fieldName
+        int diff = searchNode.getStart() - searchNode.getBytesToStoreLength();
+        // since hookup happens on a new fieldName, hence start will be bytesToStoreLength
+        searchNode.setIndex(fieldNames.size(), bytesToStoreLength + diff, searchNode.getLength(), bytesToStoreLength);
+        searchNode.setIsEndOfField(true);
+        fieldNames.add(fieldName);
+        return searchNode.getIndex();
+    }
+
+    public void serialize(DataOutput out) throws IOException {
+        out.writeInt(VERSION);
+
+        // serialize fieldNames
+        out.writeInt(fieldNames.size());
+        for (IValueReference fieldName : fieldNames) {
+            out.writeInt(fieldName.getLength());
+            out.write(fieldName.getByteArray(), fieldName.getStartOffset(), fieldName.getLength());
+        }
+
+        rootNode.serialize(out);
+    }
+
+    public List<IValueReference> getFieldNames() {
+        return fieldNames;
+    }
+
+    public IValueReference getFieldName(int fieldIndex) {
+        return fieldNames.get(fieldIndex);
+    }
+
+    public void clear() {
+        rootNode = null;
+        fieldNames.clear();
+    }
+
+    public static FieldNameTrie deserialize(DataInput in) throws IOException {
+        int version = in.readInt();
+        if (version == VERSION) {
+            return deserializeV1(in);
+        }
+        throw new IllegalStateException("Unsupported version: " + version);
+    }
+
+    private static FieldNameTrie deserializeV1(DataInput in) throws IOException {
+        int numberOfFieldNames = in.readInt();
+
+        List<IValueReference> fieldNames = new ArrayList<>();
+        deserializeFieldNames(in, fieldNames, numberOfFieldNames);
+
+        FieldNameTrie newTrie = new FieldNameTrie(fieldNames);
+        newTrie.rootNode = TrieNode.deserialize(in);
+
+        return newTrie;
+    }
+
+    @Override
+    public String toString() {
+        TrieNode currentNode = rootNode;
+        Queue<TrieNode> queue = new ArrayDeque<>();
+        currentNode.getChildren().addAllChildren(queue);
+        StringBuilder treeBuilder = new StringBuilder();
+        while (!queue.isEmpty()) {
+            int len = queue.size();
+            for (int i = 0; i < len; i++) {
+                TrieNode node = queue.poll();
+                assert node != null;
+                byte[] bytes = fieldNames.get(node.getIndex()).getByteArray();
+                for (int j = 0; j < node.getLength(); j++) {
+                    treeBuilder.append((char) bytes[node.getStart() + j]);
+                }
+                treeBuilder.append("(").append(node.isEndOfField()).append(")");
+                if (i != len - 1) {
+                    treeBuilder.append(" | ");
+                }
+
+                node.getChildren().addAllChildren(queue);
+            }
+            treeBuilder.append("\n");
+        }
+        return treeBuilder.toString();
+    }
+
+    private static class LookupState {
+        private TrieNode lastNode;
+        private int relativeOffsetFromStart;
+        private int fieldLength;
+
+        public void setState(TrieNode lastNode, int startIndex, int continuationByteIndex, int fieldLength) {
+            this.lastNode = lastNode;
+            this.relativeOffsetFromStart = continuationByteIndex - startIndex;
+            this.fieldLength = fieldLength;
+        }
+
+        public TrieNode getLastNode() {
+            return lastNode;
+        }
+
+        public int getRelativeOffsetFromStart() {
+            return relativeOffsetFromStart;
+        }
+
+        public int getFieldLength() {
+            return fieldLength;
+        }
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/dictionary/FieldNamesHashDictionary.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/dictionary/FieldNamesHashDictionary.java
new file mode 100644
index 0000000..591f322
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/dictionary/FieldNamesHashDictionary.java
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.dictionary;
+
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+import it.unimi.dsi.fastutil.ints.Int2IntMap;
+import it.unimi.dsi.fastutil.ints.Int2IntOpenHashMap;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
+
+/**
+ * @deprecated Use {@link FieldNamesTrieDictionary}
+ */
+@Deprecated
+public class FieldNamesHashDictionary extends AbstractFieldNamesDictionary {
+    //For both declared and inferred fields
+    private final List<IValueReference> fieldNames;
+    private final Object2IntMap<String> declaredFieldNamesToIndexMap;
+    private final Int2IntMap hashToFieldNameIndexMap;
+    private final IBinaryHashFunction fieldNameHashFunction;
+
+    //For lookups
+    private final ArrayBackedValueStorage lookupStorage;
+
+    public FieldNamesHashDictionary() {
+        this(new ArrayList<>(), new Object2IntOpenHashMap<>(), new Int2IntOpenHashMap());
+    }
+
+    private FieldNamesHashDictionary(List<IValueReference> fieldNames,
+            Object2IntMap<String> declaredFieldNamesToIndexMap, Int2IntMap hashToFieldNameIndexMap) {
+        super();
+        this.fieldNames = fieldNames;
+        this.declaredFieldNamesToIndexMap = declaredFieldNamesToIndexMap;
+        this.hashToFieldNameIndexMap = hashToFieldNameIndexMap;
+        fieldNameHashFunction =
+                new PointableBinaryHashFunctionFactory(UTF8StringPointable.FACTORY).createBinaryHashFunction();
+        lookupStorage = new ArrayBackedValueStorage();
+    }
+
+    @Override
+    public List<IValueReference> getFieldNames() {
+        return fieldNames;
+    }
+
+    //TODO solve collision (they're so rare that I haven't seen any)
+    @Override
+    public int getOrCreateFieldNameIndex(IValueReference fieldName) throws HyracksDataException {
+        if (fieldName == DUMMY_FIELD_NAME) {
+            return DUMMY_FIELD_NAME_INDEX;
+        }
+
+        int hash = getHash(fieldName);
+        if (!hashToFieldNameIndexMap.containsKey(hash)) {
+            int index = addFieldName(creatFieldName(fieldName), hash);
+            hashToFieldNameIndexMap.put(hash, index);
+            return index;
+        }
+        return hashToFieldNameIndexMap.get(hash);
+    }
+
+    @Override
+    public int getOrCreateFieldNameIndex(String fieldName) throws HyracksDataException {
+        if (!declaredFieldNamesToIndexMap.containsKey(fieldName)) {
+            IValueReference serializedFieldName = creatFieldName(fieldName);
+            int hash = getHash(serializedFieldName);
+            int index = addFieldName(serializedFieldName, hash);
+            declaredFieldNamesToIndexMap.put(fieldName, index);
+            return index;
+        }
+        return declaredFieldNamesToIndexMap.getInt(fieldName);
+    }
+
+    @Override
+    public int getFieldNameIndex(String fieldName) throws HyracksDataException {
+        lookupStorage.reset();
+        serializeFieldName(fieldName, lookupStorage);
+        return hashToFieldNameIndexMap.getOrDefault(getHash(lookupStorage), -1);
+    }
+
+    private int getHash(IValueReference fieldName) throws HyracksDataException {
+        byte[] object = fieldName.getByteArray();
+        int start = fieldName.getStartOffset();
+        int length = fieldName.getLength();
+
+        return fieldNameHashFunction.hash(object, start, length);
+    }
+
+    private int addFieldName(IValueReference fieldName, int hash) {
+        int index = fieldNames.size();
+        hashToFieldNameIndexMap.put(hash, index);
+        fieldNames.add(fieldName);
+        return index;
+    }
+
+    @Override
+    public IValueReference getFieldName(int index) {
+        if (index == DUMMY_FIELD_NAME_INDEX) {
+            return DUMMY_FIELD_NAME;
+        }
+        return fieldNames.get(index);
+    }
+
+    @Override
+    public void serialize(DataOutput output) throws IOException {
+        output.writeInt(fieldNames.size());
+        for (IValueReference fieldName : fieldNames) {
+            output.writeInt(fieldName.getLength());
+            output.write(fieldName.getByteArray(), fieldName.getStartOffset(), fieldName.getLength());
+        }
+
+        output.writeInt(declaredFieldNamesToIndexMap.size());
+        for (Object2IntMap.Entry<String> declaredFieldIndex : declaredFieldNamesToIndexMap.object2IntEntrySet()) {
+            output.writeUTF(declaredFieldIndex.getKey());
+            output.writeInt(declaredFieldIndex.getIntValue());
+        }
+
+        for (Int2IntMap.Entry hashIndex : hashToFieldNameIndexMap.int2IntEntrySet()) {
+            output.writeInt(hashIndex.getIntKey());
+            output.writeInt(hashIndex.getIntValue());
+        }
+    }
+
+    public static FieldNamesHashDictionary deserialize(DataInput input) throws IOException {
+        int numberOfFieldNames = input.readInt();
+
+        List<IValueReference> fieldNames = new ArrayList<>();
+        deserializeFieldNames(input, fieldNames, numberOfFieldNames);
+
+        Object2IntMap<String> declaredFieldNamesToIndexMap = new Object2IntOpenHashMap<>();
+        deserializeDeclaredFieldNames(input, declaredFieldNamesToIndexMap);
+
+        Int2IntMap hashToFieldNameIndexMap = new Int2IntOpenHashMap();
+        deserializeHashToFieldNameIndex(input, hashToFieldNameIndexMap, numberOfFieldNames);
+
+        return new FieldNamesHashDictionary(fieldNames, declaredFieldNamesToIndexMap, hashToFieldNameIndexMap);
+    }
+
+    @Override
+    public void abort(DataInputStream input) throws IOException {
+        int numberOfFieldNames = input.readInt();
+
+        fieldNames.clear();
+        deserializeFieldNames(input, fieldNames, numberOfFieldNames);
+
+        declaredFieldNamesToIndexMap.clear();
+        deserializeDeclaredFieldNames(input, declaredFieldNamesToIndexMap);
+
+        hashToFieldNameIndexMap.clear();
+        deserializeHashToFieldNameIndex(input, hashToFieldNameIndexMap, numberOfFieldNames);
+    }
+
+    private static void deserializeDeclaredFieldNames(DataInput input,
+            Object2IntMap<String> declaredFieldNamesToIndexMap) throws IOException {
+        int numberOfDeclaredFieldNames = input.readInt();
+        for (int i = 0; i < numberOfDeclaredFieldNames; i++) {
+            String fieldName = input.readUTF();
+            int fieldNameIndex = input.readInt();
+            declaredFieldNamesToIndexMap.put(fieldName, fieldNameIndex);
+        }
+    }
+
+    private static void deserializeHashToFieldNameIndex(DataInput input, Int2IntMap hashToFieldNameIndexMap,
+            int numberOfFieldNames) throws IOException {
+        for (int i = 0; i < numberOfFieldNames; i++) {
+            int hash = input.readInt();
+            int fieldNameIndex = input.readInt();
+            hashToFieldNameIndexMap.put(hash, fieldNameIndex);
+        }
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/dictionary/FieldNamesTrieDictionary.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/dictionary/FieldNamesTrieDictionary.java
new file mode 100644
index 0000000..30e10d6
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/dictionary/FieldNamesTrieDictionary.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.dictionary;
+
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+public class FieldNamesTrieDictionary extends AbstractFieldNamesDictionary {
+    private FieldNameTrie dictionary;
+    //For lookups
+    private final ArrayBackedValueStorage lookupStorage;
+
+    public FieldNamesTrieDictionary() {
+        this(new FieldNameTrie());
+    }
+
+    private FieldNamesTrieDictionary(FieldNameTrie dictionary) {
+        super();
+        this.dictionary = dictionary;
+        lookupStorage = new ArrayBackedValueStorage();
+    }
+
+    @Override
+    public List<IValueReference> getFieldNames() {
+        return dictionary.getFieldNames();
+    }
+
+    @Override
+    public int getOrCreateFieldNameIndex(IValueReference fieldName) throws HyracksDataException {
+        if (fieldName == DUMMY_FIELD_NAME) {
+            return DUMMY_FIELD_NAME_INDEX;
+        }
+
+        return dictionary.insert(fieldName);
+    }
+
+    @Override
+    public int getOrCreateFieldNameIndex(String fieldName) throws HyracksDataException {
+        return getOrCreateFieldNameIndex(creatFieldName(fieldName));
+    }
+
+    @Override
+    public int getFieldNameIndex(String fieldName) throws HyracksDataException {
+        lookupStorage.reset();
+        serializeFieldName(fieldName, lookupStorage);
+        return dictionary.lookup(lookupStorage);
+    }
+
+    @Override
+    public IValueReference getFieldName(int index) {
+        if (index == DUMMY_FIELD_NAME_INDEX) {
+            return DUMMY_FIELD_NAME;
+        }
+        return dictionary.getFieldName(index);
+    }
+
+    @Override
+    public void serialize(DataOutput output) throws IOException {
+        dictionary.serialize(output);
+    }
+
+    public static FieldNamesTrieDictionary deserialize(DataInput input) throws IOException {
+        FieldNameTrie fieldNameTrie = FieldNameTrie.deserialize(input);
+        return new FieldNamesTrieDictionary(fieldNameTrie);
+    }
+
+    @Override
+    public void abort(DataInputStream input) throws IOException {
+        dictionary.clear();
+        dictionary = FieldNameTrie.deserialize(input);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/dictionary/IFieldNamesDictionary.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/dictionary/IFieldNamesDictionary.java
new file mode 100644
index 0000000..ec494fa
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/dictionary/IFieldNamesDictionary.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.dictionary;
+
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+/**
+ * methods for defining the fieldName dictionary
+ * which is used to encode a fieldName to an index.
+ */
+public interface IFieldNamesDictionary {
+    /**
+     * @return get all the inserted field names
+     */
+    List<IValueReference> getFieldNames();
+
+    /**
+     * @param fieldName fieldName byte array
+     * @return returns index if field exist, otherwise insert fieldName and return the new index
+     * @throws HyracksDataException
+     */
+    int getOrCreateFieldNameIndex(IValueReference fieldName) throws HyracksDataException;
+
+    /**
+     * @param fieldName fieldName string
+     * @return returns index if field exist, otherwise insert fieldName and return the new index
+     * @throws HyracksDataException
+     */
+    int getOrCreateFieldNameIndex(String fieldName) throws HyracksDataException;
+
+    /**
+     * @param fieldName
+     * @return index of the field if exists otherwise -1
+     * @throws HyracksDataException
+     */
+    int getFieldNameIndex(String fieldName) throws HyracksDataException;
+
+    /**
+     * @param index encoded index
+     * @return the fieldName present at the requested index
+     */
+    IValueReference getFieldName(int index);
+
+    /**
+     * serialize the dictionary
+     * @param output
+     * @throws IOException
+     */
+    void serialize(DataOutput output) throws IOException;
+
+    /**
+     * resetting and rebuilding the dictionary
+     * @param input
+     * @throws IOException
+     */
+    void abort(DataInputStream input) throws IOException;
+
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/dictionary/TrieNode.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/dictionary/TrieNode.java
new file mode 100644
index 0000000..9ea2978
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/dictionary/TrieNode.java
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.dictionary;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hyracks.data.std.api.IValueReference;
+
+class TrieNode {
+    public static final int NOT_FOUND_INDEX = -1;
+
+    private ByteToNodeMap children;
+    private boolean isEndOfField;
+    private int index;
+    private int start; // includes the edges' byte
+    private int length; // includes the edges' byte
+    private int bytesToStoreLength;
+
+    TrieNode() {
+        this.children = new ByteToNodeMap();
+        index = NOT_FOUND_INDEX;
+    }
+
+    TrieNode(ByteToNodeMap children) {
+        this.children = children;
+        index = NOT_FOUND_INDEX;
+    }
+
+    public void setIndex(int index, int start, int length, int bytesToStoreLength) {
+        this.index = index;
+        this.start = start;
+        this.length = length;
+        this.bytesToStoreLength = bytesToStoreLength;
+    }
+
+    public void setIsEndOfField(boolean isEndOfField) {
+        this.isEndOfField = isEndOfField;
+    }
+
+    public TrieNode getChild(byte key) {
+        return children.get(key);
+    }
+
+    public void putChild(byte key, TrieNode child) {
+        children.put(key, child);
+    }
+
+    public ByteToNodeMap getChildren() {
+        return children;
+    }
+
+    public int getIndex() {
+        return index;
+    }
+
+    public int getStart() {
+        return start;
+    }
+
+    public int getLength() {
+        return length;
+    }
+
+    public int getBytesToStoreLength() {
+        return bytesToStoreLength;
+    }
+
+    public boolean isEndOfField() {
+        return isEndOfField;
+    }
+
+    public void reset() {
+        // since this object went to the new node.
+        children = new ByteToNodeMap();
+    }
+
+    public void split(IValueReference fieldName, int splitIndex) {
+        byte[] storedFieldBytes = fieldName.getByteArray();
+        byte splitByte = storedFieldBytes[start + splitIndex];
+        // something to be split, have to create a new node
+        // and do the linking.
+        TrieNode childNode = new TrieNode(children);
+
+        int leftToSplit = length - splitIndex;
+        childNode.setIndex(index, start + splitIndex, leftToSplit, bytesToStoreLength);
+        childNode.setIsEndOfField(isEndOfField);
+        // update the current search node
+        // new length would be 'c'
+        reset();
+        setIndex(index, start, splitIndex, bytesToStoreLength);
+        putChild(splitByte, childNode);
+        // since there was a split in searchNode, hence isEndOfField will be false.
+        setIsEndOfField(false);
+    }
+
+    public void serialize(DataOutput out) throws IOException {
+        // Serialize child first
+        children.serialize(out);
+        // serialize fields
+        out.writeBoolean(isEndOfField);
+        out.writeInt(index);
+        out.writeInt(start);
+        out.writeInt(length);
+        out.writeInt(bytesToStoreLength);
+    }
+
+    public static TrieNode deserialize(DataInput in) throws IOException {
+        ByteToNodeMap children = ByteToNodeMap.deserialize(in);
+        TrieNode node = new TrieNode(children);
+        node.isEndOfField = in.readBoolean();
+        node.index = in.readInt();
+        node.start = in.readInt();
+        node.length = in.readInt();
+        node.bytesToStoreLength = in.readInt();
+
+        return node;
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
index 57b842a..27f2160 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
@@ -82,6 +82,7 @@
 import org.apache.asterix.om.typecomputer.impl.CollectionToSequenceTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.ConcatNonNullTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.ConcatTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.CountNTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.DoubleIfTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.FieldAccessByIndexResultType;
 import org.apache.asterix.om.typecomputer.impl.FieldAccessByNameResultType;
@@ -417,6 +418,7 @@
     public static final FunctionIdentifier LISTIFY = FunctionConstants.newAsterix("listify", 1);
     public static final FunctionIdentifier AVG = FunctionConstants.newAsterix("agg-avg", 1);
     public static final FunctionIdentifier COUNT = FunctionConstants.newAsterix("agg-count", 1);
+    public static final FunctionIdentifier COUNTN = FunctionConstants.newAsterix("agg-countn", 1);
     public static final FunctionIdentifier SUM = FunctionConstants.newAsterix("agg-sum", 1);
     public static final FunctionIdentifier LOCAL_SUM = FunctionConstants.newAsterix("agg-local-sum", 1);
     public static final FunctionIdentifier INTERMEDIATE_SUM = FunctionConstants.newAsterix("agg-intermediate-sum", 1);
@@ -482,6 +484,7 @@
     public static final FunctionIdentifier SCALAR_ARRAYAGG = FunctionConstants.newAsterix("arrayagg", 1);
     public static final FunctionIdentifier SCALAR_AVG = FunctionConstants.newAsterix("avg", 1);
     public static final FunctionIdentifier SCALAR_COUNT = FunctionConstants.newAsterix("count", 1);
+    public static final FunctionIdentifier SCALAR_COUNTN = FunctionConstants.newAsterix("countn", 1);
     public static final FunctionIdentifier SCALAR_SUM = FunctionConstants.newAsterix("sum", 1);
     public static final FunctionIdentifier SCALAR_MAX = FunctionConstants.newAsterix("max", 1);
     public static final FunctionIdentifier SCALAR_MIN = FunctionConstants.newAsterix("min", 1);
@@ -501,6 +504,7 @@
     // serializable aggregate functions
     public static final FunctionIdentifier SERIAL_AVG = FunctionConstants.newAsterix("avg-serial", 1);
     public static final FunctionIdentifier SERIAL_COUNT = FunctionConstants.newAsterix("count-serial", 1);
+    public static final FunctionIdentifier SERIAL_COUNTN = FunctionConstants.newAsterix("countn-serial", 1);
     public static final FunctionIdentifier SERIAL_SUM = FunctionConstants.newAsterix("sum-serial", 1);
     public static final FunctionIdentifier SERIAL_LOCAL_SUM = FunctionConstants.newAsterix("local-sum-serial", 1);
     public static final FunctionIdentifier SERIAL_INTERMEDIATE_SUM =
@@ -559,6 +563,8 @@
             FunctionConstants.newAsterix("arrayagg-distinct", 1);
     public static final FunctionIdentifier COUNT_DISTINCT = FunctionConstants.newAsterix("agg-count-distinct", 1);
     public static final FunctionIdentifier SCALAR_COUNT_DISTINCT = FunctionConstants.newAsterix("count-distinct", 1);
+    public static final FunctionIdentifier COUNTN_DISTINCT = FunctionConstants.newAsterix("agg-countn-distinct", 1);
+    public static final FunctionIdentifier SCALAR_COUNTN_DISTINCT = FunctionConstants.newAsterix("countn-distinct", 1);
     public static final FunctionIdentifier SUM_DISTINCT = FunctionConstants.newAsterix("agg-sum-distinct", 1);
     public static final FunctionIdentifier SCALAR_SUM_DISTINCT = FunctionConstants.newAsterix("sum-distinct", 1);
     public static final FunctionIdentifier AVG_DISTINCT = FunctionConstants.newAsterix("agg-avg-distinct", 1);
@@ -593,6 +599,7 @@
     public static final FunctionIdentifier INTERMEDIATE_SQL_AVG =
             FunctionConstants.newAsterix("intermediate-agg-sql-avg", 1);
     public static final FunctionIdentifier SQL_COUNT = FunctionConstants.newAsterix("agg-sql-count", 1);
+    public static final FunctionIdentifier SQL_COUNTN = FunctionConstants.newAsterix("agg-sql-countn", 1);
     public static final FunctionIdentifier SQL_SUM = FunctionConstants.newAsterix("agg-sql-sum", 1);
     public static final FunctionIdentifier LOCAL_SQL_SUM = FunctionConstants.newAsterix("agg-local-sql-sum", 1);
     public static final FunctionIdentifier INTERMEDIATE_SQL_SUM =
@@ -666,6 +673,7 @@
 
     public static final FunctionIdentifier SCALAR_SQL_AVG = FunctionConstants.newAsterix("sql-avg", 1);
     public static final FunctionIdentifier SCALAR_SQL_COUNT = FunctionConstants.newAsterix("sql-count", 1);
+    public static final FunctionIdentifier SCALAR_SQL_COUNTN = FunctionConstants.newAsterix("sql-countn", 1);
     public static final FunctionIdentifier SCALAR_SQL_SUM = FunctionConstants.newAsterix("sql-sum", 1);
     public static final FunctionIdentifier SCALAR_SQL_MAX = FunctionConstants.newAsterix("sql-max", 1);
     public static final FunctionIdentifier SCALAR_SQL_MIN = FunctionConstants.newAsterix("sql-min", 1);
@@ -681,6 +689,7 @@
     // serializable sql aggregate functions
     public static final FunctionIdentifier SERIAL_SQL_AVG = FunctionConstants.newAsterix("sql-avg-serial", 1);
     public static final FunctionIdentifier SERIAL_SQL_COUNT = FunctionConstants.newAsterix("sql-count-serial", 1);
+    public static final FunctionIdentifier SERIAL_SQL_COUNTN = FunctionConstants.newAsterix("sql-countn-serial", 1);
     public static final FunctionIdentifier SERIAL_SQL_SUM = FunctionConstants.newAsterix("sql-sum-serial", 1);
     public static final FunctionIdentifier SERIAL_LOCAL_SQL_SUM =
             FunctionConstants.newAsterix("local-sql-sum-serial", 1);
@@ -744,6 +753,10 @@
             FunctionConstants.newAsterix("agg-sql-count-distinct", 1);
     public static final FunctionIdentifier SCALAR_SQL_COUNT_DISTINCT =
             FunctionConstants.newAsterix("sql-count-distinct", 1);
+    public static final FunctionIdentifier SQL_COUNTN_DISTINCT =
+            FunctionConstants.newAsterix("agg-sql-countn-distinct", 1);
+    public static final FunctionIdentifier SCALAR_SQL_COUNTN_DISTINCT =
+            FunctionConstants.newAsterix("sql-countn-distinct", 1);
     public static final FunctionIdentifier SQL_SUM_DISTINCT = FunctionConstants.newAsterix("agg-sql-sum-distinct", 1);
     public static final FunctionIdentifier SCALAR_SQL_SUM_DISTINCT =
             FunctionConstants.newAsterix("sql-sum-distinct", 1);
@@ -1575,6 +1588,7 @@
         addPrivateFunction(GLOBAL_MIN, MinMaxAggTypeComputer.INSTANCE, true);
         addPrivateFunction(NON_EMPTY_STREAM, ABooleanTypeComputer.INSTANCE, true);
         addFunction(COUNT, AInt64TypeComputer.INSTANCE, true);
+        addFunction(COUNTN, CountNTypeComputer.INSTANCE, true);
         addPrivateFunction(LOCAL_AVG, LocalAvgTypeComputer.INSTANCE, true);
         addFunction(AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(GLOBAL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
@@ -1623,11 +1637,13 @@
 
         addPrivateFunction(SERIAL_SQL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(SERIAL_SQL_COUNT, AInt64TypeComputer.INSTANCE, true);
+        addPrivateFunction(SERIAL_SQL_COUNTN, CountNTypeComputer.INSTANCE, true);
         addPrivateFunction(SERIAL_GLOBAL_SQL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(SERIAL_LOCAL_SQL_AVG, LocalAvgTypeComputer.INSTANCE, true);
         addPrivateFunction(SERIAL_INTERMEDIATE_SQL_AVG, LocalAvgTypeComputer.INSTANCE, true);
         addFunction(SCALAR_AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addFunction(SCALAR_COUNT, AInt64TypeComputer.INSTANCE, true);
+        addFunction(SCALAR_COUNTN, CountNTypeComputer.INSTANCE, true);
         addFunction(SCALAR_MAX, scalarMinMaxTypeComputer, true);
         addFunction(SCALAR_MIN, scalarMinMaxTypeComputer, true);
         addPrivateFunction(INTERMEDIATE_AVG, LocalAvgTypeComputer.INSTANCE, true);
@@ -1685,6 +1701,7 @@
         addPrivateFunction(LOCAL_SQL_AVG, LocalAvgTypeComputer.INSTANCE, true);
         addPrivateFunction(INTERMEDIATE_SQL_AVG, LocalAvgTypeComputer.INSTANCE, true);
         addFunction(SQL_COUNT, AInt64TypeComputer.INSTANCE, true);
+        addFunction(SQL_COUNTN, CountNTypeComputer.INSTANCE, true);
         addFunction(SQL_MAX, MinMaxAggTypeComputer.INSTANCE, true);
         addPrivateFunction(LOCAL_SQL_MAX, MinMaxAggTypeComputer.INSTANCE, true);
         addPrivateFunction(INTERMEDIATE_SQL_MAX, MinMaxAggTypeComputer.INSTANCE, true);
@@ -1695,6 +1712,7 @@
         addPrivateFunction(GLOBAL_SQL_MIN, MinMaxAggTypeComputer.INSTANCE, true);
         addFunction(SCALAR_SQL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addFunction(SCALAR_SQL_COUNT, AInt64TypeComputer.INSTANCE, true);
+        addFunction(SCALAR_SQL_COUNTN, CountNTypeComputer.INSTANCE, true);
         addFunction(SCALAR_SQL_MAX, scalarMinMaxTypeComputer, true);
         addFunction(SCALAR_SQL_MIN, scalarMinMaxTypeComputer, true);
         addPrivateFunction(INTERMEDIATE_SQL_AVG, LocalAvgTypeComputer.INSTANCE, true);
@@ -1743,6 +1761,7 @@
 
         addPrivateFunction(SERIAL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(SERIAL_COUNT, AInt64TypeComputer.INSTANCE, true);
+        addPrivateFunction(SERIAL_COUNTN, CountNTypeComputer.INSTANCE, true);
         addPrivateFunction(SERIAL_GLOBAL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(SERIAL_LOCAL_AVG, LocalAvgTypeComputer.INSTANCE, true);
         addPrivateFunction(SERIAL_INTERMEDIATE_AVG, LocalAvgTypeComputer.INSTANCE, true);
@@ -1782,6 +1801,11 @@
         addFunction(SQL_COUNT_DISTINCT, AInt64TypeComputer.INSTANCE, true);
         addFunction(SCALAR_SQL_COUNT_DISTINCT, AInt64TypeComputer.INSTANCE, true);
 
+        addFunction(COUNTN_DISTINCT, CountNTypeComputer.INSTANCE, true);
+        addFunction(SCALAR_COUNTN_DISTINCT, CountNTypeComputer.INSTANCE, true);
+        addFunction(SQL_COUNTN_DISTINCT, CountNTypeComputer.INSTANCE, true);
+        addFunction(SCALAR_SQL_COUNTN_DISTINCT, CountNTypeComputer.INSTANCE, true);
+
         addFunction(SUM_DISTINCT, NumericSumAggTypeComputer.INSTANCE, true);
         addFunction(SCALAR_SUM_DISTINCT, scalarNumericSumTypeComputer, true);
         addFunction(SQL_SUM_DISTINCT, NumericSumAggTypeComputer.INSTANCE, true);
@@ -2208,6 +2232,26 @@
         addDistinctAgg(COUNT_DISTINCT, COUNT);
         addScalarAgg(COUNT_DISTINCT, SCALAR_COUNT_DISTINCT);
 
+        // COUNTN
+
+        addAgg(COUNTN);
+        addLocalAgg(COUNTN, COUNTN);
+        addIntermediateAgg(COUNTN, SUM);
+        addGlobalAgg(COUNTN, SUM);
+
+        addScalarAgg(COUNTN, SCALAR_COUNTN);
+
+        addSerialAgg(COUNTN, SERIAL_COUNTN);
+        addAgg(SERIAL_COUNTN);
+        addLocalAgg(SERIAL_COUNTN, SERIAL_COUNTN);
+        addIntermediateAgg(SERIAL_COUNTN, SERIAL_SUM);
+        addGlobalAgg(SERIAL_COUNTN, SERIAL_SUM);
+
+        // COUNTN DISTINCT
+
+        addDistinctAgg(COUNTN_DISTINCT, COUNTN);
+        addScalarAgg(COUNTN_DISTINCT, SCALAR_COUNTN_DISTINCT);
+
         // MAX
         addAgg(MAX);
         addAgg(LOCAL_MAX);
@@ -2719,6 +2763,26 @@
         addDistinctAgg(SQL_COUNT_DISTINCT, SQL_COUNT);
         addScalarAgg(SQL_COUNT_DISTINCT, SCALAR_SQL_COUNT_DISTINCT);
 
+        // SQL COUNTN
+
+        addAgg(SQL_COUNTN);
+        addLocalAgg(SQL_COUNTN, SQL_COUNTN);
+        addIntermediateAgg(SQL_COUNTN, SQL_SUM);
+        addGlobalAgg(SQL_COUNTN, SQL_SUM);
+
+        addScalarAgg(SQL_COUNTN, SCALAR_SQL_COUNTN);
+
+        addSerialAgg(SQL_COUNTN, SERIAL_SQL_COUNTN);
+        addAgg(SERIAL_SQL_COUNTN);
+        addLocalAgg(SERIAL_SQL_COUNTN, SERIAL_SQL_COUNTN);
+        addIntermediateAgg(SERIAL_SQL_COUNTN, SERIAL_SQL_SUM);
+        addGlobalAgg(SERIAL_SQL_COUNTN, SERIAL_SQL_SUM);
+
+        // SQL COUNTN DISTINCT
+
+        addDistinctAgg(SQL_COUNTN_DISTINCT, SQL_COUNTN);
+        addScalarAgg(SQL_COUNTN_DISTINCT, SCALAR_SQL_COUNTN_DISTINCT);
+
         // SQL MAX
         addAgg(SQL_MAX);
         addAgg(LOCAL_SQL_MAX);
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/ARecordVisitablePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/ARecordVisitablePointable.java
index 6637af8..440fd6f 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/ARecordVisitablePointable.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/ARecordVisitablePointable.java
@@ -22,6 +22,7 @@
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.LinkedHashSet;
 import java.util.List;
 
 import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
@@ -39,6 +40,8 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.util.string.UTF8StringWriter;
 
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+
 /**
  * This class interprets the binary data representation of a record. One can
  * call getFieldNames, getFieldTypeTags and getFieldValues to get pointable
@@ -56,6 +59,8 @@
     // access results: field names, field types, and field values
     private final List<IVisitablePointable> fieldNames = new ArrayList<>();
     private final List<IVisitablePointable> fieldValues = new ArrayList<>();
+    private final IntArrayList reverseLookupClosedFields = new IntArrayList();
+    private int numFields = 0;
 
     // pointable allocator
     private final PointableAllocator allocator = new PointableAllocator();
@@ -63,6 +68,7 @@
     private final ResettableByteArrayOutputStream typeBos = new ResettableByteArrayOutputStream();
 
     private final ResettableByteArrayOutputStream dataBos = new ResettableByteArrayOutputStream();
+
     private final DataOutputStream dataDos = new DataOutputStream(dataBos);
 
     private final ARecordType inputRecType;
@@ -91,15 +97,38 @@
         try {
             final DataOutputStream typeDos = new DataOutputStream(typeBos);
             final UTF8StringWriter utf8Writer = new UTF8StringWriter();
-            for (int i = 0; i < numberOfSchemaFields; i++) {
-                // add type name Reference (including a string type tag)
-                int nameStart = typeBos.size();
-                typeDos.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
-                utf8Writer.writeUTF8(fieldNameStrs[i], typeDos);
-                int nameEnd = typeBos.size();
-                IVisitablePointable typeNameReference = AFlatValuePointable.FACTORY.create(null);
-                typeNameReference.set(typeBos.getByteArray(), nameStart, nameEnd - nameStart);
-                fieldNames.add(typeNameReference);
+            LinkedHashSet<String> allOrderedFields = inputType.getAllOrderedFields();
+            if (allOrderedFields != null) {
+                numFields = allOrderedFields.size();
+                int index = 0;
+                int nameInClosedField = 0;
+                for (String field : allOrderedFields) {
+                    int nameStart = typeBos.size();
+                    typeDos.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+                    utf8Writer.writeUTF8(field, typeDos);
+                    int nameEnd = typeBos.size();
+                    IVisitablePointable typeNameReference = AFlatValuePointable.FACTORY.create(null);
+                    typeNameReference.set(typeBos.getByteArray(), nameStart, nameEnd - nameStart);
+                    fieldNames.add(typeNameReference);
+                    fieldValues.add(missingReference);
+                    if (nameInClosedField < numberOfSchemaFields && field.equals(fieldNameStrs[nameInClosedField])) {
+                        reverseLookupClosedFields.add(index);
+                        nameInClosedField++;
+                    }
+                    index++;
+                }
+            } else {
+                for (int i = 0; i < numberOfSchemaFields; i++) {
+                    // add type name Reference (including a string type tag)
+                    int nameStart = typeBos.size();
+                    typeDos.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+                    utf8Writer.writeUTF8(fieldNameStrs[i], typeDos);
+                    int nameEnd = typeBos.size();
+                    IVisitablePointable typeNameReference = AFlatValuePointable.FACTORY.create(null);
+                    typeNameReference.set(typeBos.getByteArray(), nameStart, nameEnd - nameStart);
+                    fieldNames.add(typeNameReference);
+                    reverseLookupClosedFields.add(fieldNames.size() - 1);
+                }
             }
 
             // initialize a constant: null value bytes reference
@@ -113,6 +142,7 @@
             typeDos.writeByte(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
             int missingFieldEnd = typeBos.size();
             missingReference.set(typeBos.getByteArray(), missingFieldStart, missingFieldEnd - missingFieldStart);
+
         } catch (IOException e) {
             throw new IllegalStateException(e);
         }
@@ -126,11 +156,22 @@
         // reset the allocator
         allocator.reset();
 
+        int removeTill = numFields;
+        if (numFields == 0) {
+            removeTill = numberOfSchemaFields;
+        }
         // clean up the returned containers
-        for (int i = fieldNames.size() - 1; i >= numberOfSchemaFields; i--) {
+        for (int i = fieldNames.size() - 1; i >= removeTill; i--) {
             fieldNames.remove(i);
         }
-        fieldValues.clear();
+
+        for (int i = fieldValues.size() - 1; i >= numFields; i--) {
+            fieldValues.remove(i);
+        }
+
+        for (int i = 0; i < numFields; i++) {
+            fieldValues.set(i, missingReference);
+        }
     }
 
     @Override
@@ -176,16 +217,25 @@
                     offsetArrayOffset += 4;
                 }
                 for (int fieldNumber = 0; fieldNumber < numberOfSchemaFields; fieldNumber++) {
+                    int index = reverseLookupClosedFields.get(fieldNumber);
                     if (hasOptionalFields) {
                         byte b1 = b[nullBitMapOffset + fieldNumber / 4];
                         if (RecordUtil.isNull(b1, fieldNumber)) {
                             // set null value (including type tag inside)
-                            fieldValues.add(nullReference);
+                            if (index < numFields) {
+                                fieldValues.set(index, nullReference);
+                            } else {
+                                fieldValues.add(nullReference);
+                            }
                             continue;
                         }
                         if (RecordUtil.isMissing(b1, fieldNumber)) {
                             // set missing value (including type tag inside)
-                            fieldValues.add(missingReference);
+                            if (index < numFields) {
+                                fieldValues.set(index, missingReference);
+                            } else {
+                                fieldValues.add(missingReference);
+                            }
                             continue;
                         }
                     }
@@ -212,12 +262,18 @@
                     int fend = dataBos.size();
                     IVisitablePointable fieldValue = allocator.allocateFieldValue(fieldType);
                     fieldValue.set(dataBos.getByteArray(), fstart, fend - fstart);
-                    fieldValues.add(fieldValue);
+                    if (index < numFields) {
+                        fieldValues.set(index, fieldValue);
+                    } else {
+                        fieldValues.add(fieldValue);
+                    }
                 }
             }
             if (isExpanded) {
                 int numberOfOpenFields = AInt32SerializerDeserializer.getInt(b, openPartOffset);
                 int fieldOffset = openPartOffset + 4 + (8 * numberOfOpenFields);
+                int currentCheck = 0;
+                int reverseLookupIndex = 0;
                 for (int i = 0; i < numberOfOpenFields; i++) {
                     // set the field name (including a type tag, which is a string)
                     int fieldValueLength =
@@ -228,7 +284,22 @@
                     int fnend = dataBos.size();
                     IVisitablePointable fieldName = allocator.allocateEmpty();
                     fieldName.set(dataBos.getByteArray(), fnstart, fnend - fnstart);
-                    fieldNames.add(fieldName);
+                    boolean addItToNumFields = true;
+                    for (; currentCheck < numFields; currentCheck++) {
+                        if (reverseLookupIndex < reverseLookupClosedFields.size()
+                                && currentCheck == reverseLookupClosedFields.get(reverseLookupIndex)) {
+                            reverseLookupIndex++;
+                            continue;
+                        }
+
+                        if (fieldNames.get(currentCheck).equals(fieldName)) {
+                            break;
+                        }
+                    }
+                    if (currentCheck >= numFields) {
+                        addItToNumFields = false;
+                        fieldNames.add(fieldName);
+                    }
                     fieldOffset += fieldValueLength;
 
                     typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(b[fieldOffset]);
@@ -238,7 +309,12 @@
                     // allocate
                     IVisitablePointable fieldValueAccessor = allocator.allocateFieldValue(typeTag, b, fieldOffset + 1);
                     fieldValueAccessor.set(b, fieldOffset, fieldValueLength);
-                    fieldValues.add(fieldValueAccessor);
+                    if (!addItToNumFields) {
+                        fieldValues.add(fieldValueAccessor);
+                    } else {
+                        fieldValues.set(currentCheck, fieldValueAccessor);
+                        currentCheck++;
+                    }
                     fieldOffset += fieldValueLength;
                 }
             }
@@ -263,5 +339,4 @@
     public <R, T> R accept(IVisitablePointableVisitor<R, T> vistor, T tag) throws HyracksDataException {
         return vistor.visit(this, tag);
     }
-
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/ARecordPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/ARecordPrinter.java
index 7823637..9aaf889 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/ARecordPrinter.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/ARecordPrinter.java
@@ -33,13 +33,13 @@
  * This class is to print the content of a record.
  */
 public class ARecordPrinter {
-    private final String startRecord;
-    private final String endRecord;
-    private final String fieldSeparator;
-    private final String fieldNameSeparator;
+    protected final String startRecord;
+    protected final String endRecord;
+    protected final String fieldSeparator;
+    protected final String fieldNameSeparator;
 
-    private final Pair<PrintStream, ATypeTag> nameVisitorArg = new Pair<>(null, ATypeTag.STRING);
-    private final Pair<PrintStream, ATypeTag> itemVisitorArg = new Pair<>(null, null);
+    protected final Pair<PrintStream, ATypeTag> nameVisitorArg = new Pair<>(null, ATypeTag.STRING);
+    protected final Pair<PrintStream, ATypeTag> itemVisitorArg = new Pair<>(null, null);
 
     public ARecordPrinter(final String startRecord, final String endRecord, final String fieldSeparator,
             final String fieldNameSeparator) {
@@ -59,8 +59,8 @@
 
         ps.print(startRecord);
 
-        final int size = fieldNames.size();
         boolean first = true;
+        final int size = fieldNames.size();
         for (int i = 0; i < size; ++i) {
             final IVisitablePointable fieldName = fieldNames.get(i);
             final IVisitablePointable fieldValue = fieldValues.get(i);
@@ -82,7 +82,7 @@
         ps.print(endRecord);
     }
 
-    private void printField(PrintStream ps, IPrintVisitor visitor, IVisitablePointable fieldName,
+    protected void printField(PrintStream ps, IPrintVisitor visitor, IVisitablePointable fieldName,
             IVisitablePointable fieldValue, ATypeTag fieldTypeTag) throws HyracksDataException {
         itemVisitorArg.second = fieldTypeTag;
         if (fieldNameSeparator != null) {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/csv/ACSVRecordPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/csv/ACSVRecordPrinter.java
new file mode 100644
index 0000000..ce9e3a3
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/csv/ACSVRecordPrinter.java
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.om.pointables.printer.csv;
+
+import static org.apache.asterix.om.types.hierachy.ATypeHierarchy.isCompatible;
+
+import java.io.PrintStream;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.pointables.printer.ARecordPrinter;
+import org.apache.asterix.om.pointables.printer.IPrintVisitor;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.util.string.UTF8StringUtil;
+
+public class ACSVRecordPrinter extends ARecordPrinter {
+    private ARecordType schema;
+    private boolean firstRecord;
+    private boolean header;
+    private final String recordDelimiter;
+    private static final List<ATypeTag> supportedTypes = List.of(ATypeTag.TINYINT, ATypeTag.SMALLINT, ATypeTag.INTEGER,
+            ATypeTag.BIGINT, ATypeTag.UINT8, ATypeTag.UINT16, ATypeTag.UINT64, ATypeTag.FLOAT, ATypeTag.DOUBLE,
+            ATypeTag.STRING, ATypeTag.BOOLEAN, ATypeTag.DATETIME, ATypeTag.UINT32, ATypeTag.DATE, ATypeTag.TIME);
+
+    public ACSVRecordPrinter(final String startRecord, final String endRecord, final String fieldSeparator,
+            final String fieldNameSeparator, String recordDelimiter, ARecordType schema, String headerStr) {
+        super(startRecord, endRecord, fieldSeparator, fieldNameSeparator);
+        this.schema = schema;
+        this.header = headerStr != null && Boolean.parseBoolean(headerStr);
+        this.firstRecord = true;
+        this.recordDelimiter = recordDelimiter;
+    }
+
+    @Override
+    public void printRecord(ARecordVisitablePointable recordAccessor, PrintStream ps, IPrintVisitor visitor)
+            throws HyracksDataException {
+        // backward compatibility -- No Schema print it as it is from recordAccessor
+        if (schema == null) {
+            super.printRecord(recordAccessor, ps, visitor);
+        } else {
+            printSchemaFullRecord(recordAccessor, ps, visitor);
+        }
+    }
+
+    private void printSchemaFullRecord(ARecordVisitablePointable recordAccessor, PrintStream ps, IPrintVisitor visitor)
+            throws HyracksDataException {
+        // check the schema for the record
+        // try producing the record into the record of expected schema
+        Map<String, ATypeTag> schemaDetails = new HashMap<>();
+        if (checkCSVSchema(recordAccessor, schemaDetails)) {
+            nameVisitorArg.first = ps;
+            itemVisitorArg.first = ps;
+            if (header) {
+                addHeader(recordAccessor, ps, visitor);
+            }
+            // add record delimiter
+            // by default the separator between the header and the records is "\n"
+            if (firstRecord) {
+                firstRecord = false;
+            } else {
+                ps.print(recordDelimiter);
+            }
+            final List<IVisitablePointable> fieldNames = recordAccessor.getFieldNames();
+            final List<IVisitablePointable> fieldValues = recordAccessor.getFieldValues();
+
+            boolean first = true;
+            for (int i = 0; i < fieldNames.size(); ++i) {
+                final IVisitablePointable fieldNamePointable = fieldNames.get(i);
+                String fieldName = UTF8StringUtil.toString(fieldNamePointable.getByteArray(),
+                        fieldNamePointable.getStartOffset() + 1);
+                final IVisitablePointable fieldValue = fieldValues.get(i);
+                final ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
+                        .deserialize(fieldValue.getByteArray()[fieldValue.getStartOffset()]);
+                ATypeTag expectedTypeTag = schemaDetails.get(fieldName);
+                if (!isCompatible(typeTag, expectedTypeTag)) {
+                    expectedTypeTag = ATypeTag.NULL;
+                }
+                if (first) {
+                    first = false;
+                } else {
+                    ps.print(fieldSeparator);
+                }
+                printField(ps, visitor, fieldNamePointable, fieldValue, expectedTypeTag);
+            }
+        }
+    }
+
+    private boolean checkCSVSchema(ARecordVisitablePointable recordAccessor, Map<String, ATypeTag> schemaDetails) {
+        final List<IVisitablePointable> fieldNames = recordAccessor.getFieldNames();
+        final List<IVisitablePointable> fieldValues = recordAccessor.getFieldValues();
+        final List<String> expectedFieldNames = Arrays.asList(schema.getFieldNames());
+        final List<IAType> expectedFieldTypes = Arrays.asList(schema.getFieldTypes());
+        if (fieldNames.size() != expectedFieldNames.size()) {
+            // todo: raise warning about schema mismatch
+            return false;
+        }
+        for (int i = 0; i < fieldNames.size(); ++i) {
+            final IVisitablePointable fieldName = fieldNames.get(i);
+            String fieldColumnName = UTF8StringUtil.toString(fieldName.getByteArray(), fieldName.getStartOffset() + 1);
+            final IVisitablePointable fieldValue = fieldValues.get(i);
+            final ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
+                    .deserialize(fieldValue.getByteArray()[fieldValue.getStartOffset()]);
+            ATypeTag expectedType;
+            boolean canNull = false;
+            if (expectedFieldNames.contains(fieldColumnName)) {
+                IAType expectedIAType = expectedFieldTypes.get(expectedFieldNames.indexOf(fieldColumnName));
+                if (!supportedTypes.contains(expectedIAType.getTypeTag())) {
+                    if (expectedIAType.getTypeTag().equals(ATypeTag.UNION)) {
+                        AUnionType unionType = (AUnionType) expectedIAType;
+                        expectedType = unionType.getActualType().getTypeTag();
+                        canNull = unionType.isNullableType();
+                        if (!supportedTypes.contains(expectedType)) {
+                            // unsupported DataType
+                            return false;
+                        }
+                    } else {
+                        // todo: unexpected type
+                        return false;
+                    }
+                } else {
+                    expectedType = expectedIAType.getTypeTag();
+                }
+                schemaDetails.put(fieldColumnName, expectedType);
+            } else {
+                // todo: raise warning about schema mismatch
+                return false;
+            }
+            if (typeTag.equals(ATypeTag.MISSING) || (typeTag.equals(ATypeTag.NULL) && !canNull)) {
+                // todo: raise warning about schema mismatch
+                return false;
+            }
+            if (!isCompatible(typeTag, expectedType) && !canNull) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    private void addHeader(ARecordVisitablePointable recordAccessor, PrintStream ps, IPrintVisitor visitor)
+            throws HyracksDataException {
+        //check if it is a first record
+        if (firstRecord) {
+            final List<IVisitablePointable> fieldNames = recordAccessor.getFieldNames();
+            boolean first = true;
+            for (int i = 0; i < fieldNames.size(); ++i) {
+                if (first) {
+                    first = false;
+                } else {
+                    ps.print(fieldSeparator);
+                }
+                printFieldName(ps, visitor, fieldNames.get(i));
+            }
+            firstRecord = false;
+        }
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/csv/APrintVisitor.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/csv/APrintVisitor.java
index 3f22374..22c502b 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/csv/APrintVisitor.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/csv/APrintVisitor.java
@@ -19,14 +19,20 @@
 
 package org.apache.asterix.om.pointables.printer.csv;
 
+import static org.apache.asterix.dataflow.data.nontagged.printers.csv.CSVUtils.KEY_HEADER;
+import static org.apache.asterix.dataflow.data.nontagged.printers.csv.CSVUtils.KEY_RECORD_DELIMITER;
+
 import java.io.PrintStream;
+import java.util.Map;
 
 import org.apache.asterix.dataflow.data.nontagged.printers.csv.AObjectPrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.csv.CSVUtils;
 import org.apache.asterix.om.pointables.AListVisitablePointable;
 import org.apache.asterix.om.pointables.ARecordVisitablePointable;
 import org.apache.asterix.om.pointables.printer.AListPrinter;
 import org.apache.asterix.om.pointables.printer.ARecordPrinter;
 import org.apache.asterix.om.pointables.printer.AbstractPrintVisitor;
+import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -36,6 +42,15 @@
  * PrintStream in CSV format.
  */
 public class APrintVisitor extends AbstractPrintVisitor {
+    private final ARecordType itemType;
+    private final Map<String, String> configuration;
+
+    public APrintVisitor(ARecordType itemType, Map<String, String> configuration) {
+        super();
+        this.itemType = itemType;
+        this.configuration = configuration;
+    }
+
     @Override
     protected AListPrinter createListPrinter(AListVisitablePointable accessor) throws HyracksDataException {
         throw new HyracksDataException("'List' type unsupported for CSV output");
@@ -43,12 +58,15 @@
 
     @Override
     protected ARecordPrinter createRecordPrinter(ARecordVisitablePointable accessor) {
-        return new ARecordPrinter("", "", ",", null);
+        String delimiter = CSVUtils.getDelimiter(configuration);
+        String recordDelimiter = configuration.get(KEY_RECORD_DELIMITER) == null ? (itemType == null ? "" : "\n")
+                : configuration.get(KEY_RECORD_DELIMITER);
+        return new ACSVRecordPrinter("", "", delimiter, null, recordDelimiter, itemType, configuration.get(KEY_HEADER));
     }
 
     @Override
     protected boolean printFlatValue(ATypeTag typeTag, byte[] b, int s, int l, PrintStream ps)
             throws HyracksDataException {
-        return AObjectPrinterFactory.printFlatValue(typeTag, b, s, l, ps);
+        return AObjectPrinterFactory.createInstance(itemType, configuration).printFlatValue(typeTag, b, s, l, ps);
     }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/CountNTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/CountNTypeComputer.java
new file mode 100644
index 0000000..4178315
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/CountNTypeComputer.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public class CountNTypeComputer extends AbstractResultTypeComputer {
+
+    public static final CountNTypeComputer INSTANCE = new CountNTypeComputer();
+
+    private CountNTypeComputer() {
+    }
+
+    protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+        return AUnionType.createNullableType(BuiltinType.AINT64);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java
index 838f6f8..8358627 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java
@@ -24,9 +24,11 @@
 import java.util.Iterator;
 import java.util.LinkedHashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.asterix.common.annotations.RecordFieldOrderAnnotation;
+import org.apache.asterix.common.config.CompilerProperties;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
@@ -43,6 +45,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import org.apache.hyracks.algebricks.core.config.AlgebricksConfig;
 import org.apache.hyracks.util.LogRedactionUtil;
 
 public class OpenRecordConstructorResultType implements IResultTypeComputer {
@@ -62,6 +65,13 @@
             return type;
         }
 
+        boolean orderFields = AlgebricksConfig.ORDERED_FIELDS;
+        if (metadataProvider != null) {
+            Map<String, Object> config = metadataProvider.getConfig();
+            orderFields = Boolean.parseBoolean((String) config
+                    .getOrDefault(CompilerProperties.COMPILER_ORDERED_FIELDS_KEY, String.valueOf(orderFields)));
+        }
+
         Iterator<Mutable<ILogicalExpression>> argIter = f.getArguments().iterator();
         List<String> namesList = new ArrayList<>();
         List<IAType> typesList = new ArrayList<>();
@@ -102,7 +112,12 @@
         IAType[] fieldTypes = typesList.toArray(new IAType[0]);
         ARecordType resultType;
         if (isOpen && canProvideAdditionFieldInfo) {
-            resultType = new ARecordType(null, fieldNames, fieldTypes, isOpen, allPossibleAdditionalFieldNames);
+            if (orderFields) {
+                resultType = new ARecordType(null, fieldNames, fieldTypes, isOpen, allPossibleAdditionalFieldNames,
+                        allPossibleFieldNamesOrdered);
+            } else {
+                resultType = new ARecordType(null, fieldNames, fieldTypes, isOpen, allPossibleAdditionalFieldNames);
+            }
             resultType.getAnnotations().add(new RecordFieldOrderAnnotation(allPossibleFieldNamesOrdered));
         } else {
             resultType = new ARecordType(null, fieldNames, fieldTypes, isOpen);
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
index fd51433..ba352ac 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
@@ -23,6 +23,7 @@
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -54,10 +55,15 @@
 
     private static final long serialVersionUID = 1L;
     private static final JavaType SET = OBJECT_MAPPER.getTypeFactory().constructCollectionType(Set.class, String.class);
+
+    private static final JavaType LIST =
+            OBJECT_MAPPER.getTypeFactory().constructCollectionType(LinkedHashSet.class, String.class);
     private static final String IS_OPEN = "isOpen";
     private static final String FIELD_NAMES = "fieldNames";
     private static final String FIELD_TYPES = "fieldTypes";
     private static final String ADDITIONAL_FIELDS = "additionalFieldNames";
+
+    private static final String ORDERED_FIELDS = "orderedFields";
     private final String[] fieldNames;
     private final IAType[] fieldTypes;
     private final Map<String, Integer> fieldNameToIndexMap = new HashMap<>();
@@ -71,6 +77,8 @@
     // the bounded set of all possible additional field names.
     private final Set<String> allPossibleAdditionalFieldNames;
 
+    private final LinkedHashSet<String> allOrderedFields;
+
     /**
      * @param typeName   the name of the type
      * @param fieldNames the names of the closed fields
@@ -78,7 +86,31 @@
      * @param isOpen     whether the record is open
      */
     public ARecordType(String typeName, String[] fieldNames, IAType[] fieldTypes, boolean isOpen) {
-        this(typeName, fieldNames, fieldTypes, isOpen, null);
+        this(typeName, fieldNames, fieldTypes, isOpen, null, null);
+    }
+
+    /**
+     * @param typeName   the name of the type
+     * @param fieldNames the names of the closed fields
+     * @param fieldTypes the types of the closed fields
+     * @param isOpen     whether the record is open
+     * @param allPossibleAdditionalFieldNames, all possible additional field names.
+     */
+    public ARecordType(String typeName, String[] fieldNames, IAType[] fieldTypes, boolean isOpen,
+            Set<String> allPossibleAdditionalFieldNames) {
+        this(typeName, fieldNames, fieldTypes, isOpen, allPossibleAdditionalFieldNames, null);
+    }
+
+    /**
+     * @param typeName   the name of the type
+     * @param fieldNames the names of the closed fields
+     * @param fieldTypes the types of the closed fields
+     * @param isOpen     whether the record is open
+     * @param allOrderedFields fields in order.
+     */
+    public ARecordType(String typeName, String[] fieldNames, IAType[] fieldTypes, boolean isOpen,
+            LinkedHashSet<String> allOrderedFields) {
+        this(typeName, fieldNames, fieldTypes, isOpen, null, allOrderedFields);
     }
 
     /**
@@ -86,10 +118,11 @@
      * @param fieldNames                       the names of the closed fields
      * @param fieldTypes                       the types of the closed fields
      * @param isOpen                           whether the record is open
-     * @param allPossibleAdditionalFieldNames, all possible additional field names.
+     * @param allPossibleAdditionalFieldNames, all possible additional field names
+     * @param allOrderedFields fields in order.
      */
     public ARecordType(String typeName, String[] fieldNames, IAType[] fieldTypes, boolean isOpen,
-            Set<String> allPossibleAdditionalFieldNames) {
+            Set<String> allPossibleAdditionalFieldNames, LinkedHashSet<String> allOrderedFields) {
         super(typeName);
         this.fieldNames = fieldNames;
         this.fieldTypes = fieldTypes;
@@ -100,6 +133,7 @@
             fieldNameToIndexMap.put(fieldNames[index], index);
         }
         this.allPossibleAdditionalFieldNames = allPossibleAdditionalFieldNames;
+        this.allOrderedFields = allOrderedFields;
     }
 
     public boolean canContainField(String fieldName) {
@@ -383,6 +417,7 @@
         jsonObject.put(IS_OPEN, isOpen);
         jsonObject.putPOJO(FIELD_NAMES, fieldNames);
         jsonObject.putPOJO(ADDITIONAL_FIELDS, allPossibleAdditionalFieldNames);
+        jsonObject.putPOJO(ORDERED_FIELDS, allOrderedFields);
         ArrayNode fieldTypesArray = OBJECT_MAPPER.createArrayNode();
         for (int i = 0; i < fieldTypes.length; i++) {
             fieldTypesArray.add(fieldTypes[i].toJson(registry));
@@ -397,12 +432,17 @@
         boolean isOpen = json.get(IS_OPEN).asBoolean();
         String[] fieldNames = OBJECT_MAPPER.convertValue(json.get(FIELD_NAMES), String[].class);
         Set<String> additionalFields = OBJECT_MAPPER.convertValue(json.get(ADDITIONAL_FIELDS), SET);
+        LinkedHashSet<String> orderedFields = OBJECT_MAPPER.convertValue(json.get(ORDERED_FIELDS), LIST);
         ArrayNode fieldTypesNode = (ArrayNode) json.get(FIELD_TYPES);
         IAType[] fieldTypes = new IAType[fieldTypesNode.size()];
         for (int i = 0; i < fieldTypesNode.size(); i++) {
             fieldTypes[i] = (IAType) registry.deserialize(fieldTypesNode.get(i));
         }
-        return new ARecordType(typeName, fieldNames, fieldTypes, isOpen, additionalFields);
+        return new ARecordType(typeName, fieldNames, fieldTypes, isOpen, additionalFields, orderedFields);
+    }
+
+    public LinkedHashSet<String> getAllOrderedFields() {
+        return allOrderedFields;
     }
 
     public List<IAType> getFieldTypes(List<List<String>> fields) throws AlgebricksException {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/ATypeHierarchy.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/ATypeHierarchy.java
index 82de905..c94a5ae 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/ATypeHierarchy.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/ATypeHierarchy.java
@@ -177,6 +177,13 @@
         return null;
     }
 
+    /*
+    * Checks whether a type can be promoted (or converted) from one type (type1)
+    * to another (type2). This promotion typically follows certain rules defined
+    * within the class to allow safe type conversions,
+    * such as promoting a smaller numeric type (e.g., TINYINT) to a larger numeric type
+    * (e.g., INTEGER or DOUBLE), where no precision or range is lost.
+    */
     public static boolean canPromote(ATypeTag type1, ATypeTag type2) {
         return typePromotionHierachyMap.get(type1.ordinal() * ATypeTag.TYPE_COUNT + type2.ordinal());
     }
diff --git a/asterixdb/asterix-runtime/pom.xml b/asterixdb/asterix-runtime/pom.xml
index 138c4dd..b466d0d 100644
--- a/asterixdb/asterix-runtime/pom.xml
+++ b/asterixdb/asterix-runtime/pom.xml
@@ -151,8 +151,12 @@
       <artifactId>commons-collections4</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.esri.geometry</groupId>
-      <artifactId>esri-geometry-api</artifactId>
+      <groupId>org.locationtech.jts</groupId>
+      <artifactId>jts-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.parquet</groupId>
+      <artifactId>parquet-column</artifactId>
     </dependency>
   </dependencies>
 </project>
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarCountNAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarCountNAggregateDescriptor.java
new file mode 100644
index 0000000..fb2fbc8
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarCountNAggregateDescriptor.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.scalar;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.std.CountNAggregateDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class ScalarCountNAggregateDescriptor extends AbstractScalarAggregateDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final FunctionIdentifier FID = BuiltinFunctions.SCALAR_COUNTN;
+
+    public static final IFunctionDescriptorFactory FACTORY = ScalarCountNAggregateDescriptor::new;
+
+    private ScalarCountNAggregateDescriptor() {
+        super(CountNAggregateDescriptor.FACTORY);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarCountNDistinctAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarCountNDistinctAggregateDescriptor.java
new file mode 100644
index 0000000..6f4cf3a
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarCountNDistinctAggregateDescriptor.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.aggregates.scalar;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.std.CountNAggregateDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class ScalarCountNDistinctAggregateDescriptor extends AbstractScalarDistinctAggregateDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final FunctionIdentifier FID = BuiltinFunctions.SCALAR_COUNTN_DISTINCT;
+
+    public static final IFunctionDescriptorFactory FACTORY =
+            createDescriptorFactory(ScalarCountNDistinctAggregateDescriptor::new);
+
+    private ScalarCountNDistinctAggregateDescriptor() {
+        super(CountNAggregateDescriptor.FACTORY);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlCountNAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlCountNAggregateDescriptor.java
new file mode 100644
index 0000000..c0925d4
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlCountNAggregateDescriptor.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.scalar;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.std.SqlCountNAggregateDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class ScalarSqlCountNAggregateDescriptor extends AbstractScalarAggregateDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final FunctionIdentifier FID = BuiltinFunctions.SCALAR_SQL_COUNTN;
+
+    public static final IFunctionDescriptorFactory FACTORY = ScalarSqlCountNAggregateDescriptor::new;
+
+    private ScalarSqlCountNAggregateDescriptor() {
+        super(SqlCountNAggregateDescriptor.FACTORY);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlCountNDistinctAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlCountNDistinctAggregateDescriptor.java
new file mode 100644
index 0000000..a163b8f
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlCountNDistinctAggregateDescriptor.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.aggregates.scalar;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.std.SqlCountNAggregateDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class ScalarSqlCountNDistinctAggregateDescriptor extends AbstractScalarDistinctAggregateDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final FunctionIdentifier FID = BuiltinFunctions.SCALAR_SQL_COUNTN_DISTINCT;
+
+    public static final IFunctionDescriptorFactory FACTORY =
+            createDescriptorFactory(ScalarSqlCountNDistinctAggregateDescriptor::new);
+
+    private ScalarSqlCountNDistinctAggregateDescriptor() {
+        super(SqlCountNAggregateDescriptor.FACTORY);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableCountAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableCountAggregateFunction.java
index 03dacd2..1e611b9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableCountAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableCountAggregateFunction.java
@@ -81,7 +81,7 @@
         if (typeTag == ATypeTag.MISSING || typeTag == ATypeTag.NULL) {
             processNull(state, start);
         } else {
-            cnt++;
+            cnt = processValue(typeTag, cnt);
         }
         BufferSerDeUtil.writeBoolean(metNull, state, start + MET_NULL_OFFSET);
         BufferSerDeUtil.writeLong(cnt, state, start + COUNT_OFFSET);
@@ -111,4 +111,8 @@
     protected void processNull(byte[] state, int start) {
         BufferSerDeUtil.writeBoolean(true, state, start + MET_NULL_OFFSET);
     }
+
+    protected long processValue(ATypeTag tag, long cnt) {
+        return cnt + 1;
+    }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountNAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountNAggregateDescriptor.java
new file mode 100644
index 0000000..113c514
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountNAggregateDescriptor.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.serializable.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Counts the number of numeric items. Returns NULL if MISSING or NULL is encountered.
+ */
+public class SerializableCountNAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = SerializableCountNAggregateDescriptor::new;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.SERIAL_COUNTN;
+    }
+
+    @Override
+    public ISerializedAggregateEvaluatorFactory createSerializableAggregateEvaluatorFactory(
+            final IScalarEvaluatorFactory[] args) {
+        return new ISerializedAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ISerializedAggregateEvaluator createAggregateEvaluator(IEvaluatorContext ctx)
+                    throws HyracksDataException {
+                return new SerializableCountNAggregateFunction(args, ctx, sourceLoc);
+            }
+        };
+    }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountNAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountNAggregateFunction.java
new file mode 100644
index 0000000..020d367
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountNAggregateFunction.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.serializable.std;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+/**
+ * COUNTN returns the number of numeric items in the given list. Returns NULL if MISSING or NULL is encountered.
+ */
+public class SerializableCountNAggregateFunction extends AbstractSerializableCountAggregateFunction {
+    public SerializableCountNAggregateFunction(IScalarEvaluatorFactory[] args, IEvaluatorContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
+    }
+
+    @Override
+    protected long processValue(ATypeTag typeTag, long cnt) {
+        if (ATypeHierarchy.getTypeDomain(typeTag) == ATypeHierarchy.Domain.NUMERIC) {
+            return cnt + 1;
+        }
+        return cnt;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountNAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountNAggregateDescriptor.java
new file mode 100644
index 0000000..38c5150
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountNAggregateDescriptor.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.serializable.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Counts the number of numeric items. NULLs and MISSINGs are ignored.
+ */
+public class SerializableSqlCountNAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = SerializableSqlCountNAggregateDescriptor::new;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.SERIAL_SQL_COUNTN;
+    }
+
+    @Override
+    public ISerializedAggregateEvaluatorFactory createSerializableAggregateEvaluatorFactory(
+            final IScalarEvaluatorFactory[] args) {
+        return new ISerializedAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ISerializedAggregateEvaluator createAggregateEvaluator(IEvaluatorContext ctx)
+                    throws HyracksDataException {
+                return new SerializableSqlCountNAggregateFunction(args, ctx, sourceLoc);
+            }
+        };
+    }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountNAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountNAggregateFunction.java
new file mode 100644
index 0000000..41a1df3
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountNAggregateFunction.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.serializable.std;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+/**
+ * COUNTN returns the number of numeric items in the given list. NULLs and MISSINGs are ignored.
+ */
+public class SerializableSqlCountNAggregateFunction extends AbstractSerializableCountAggregateFunction {
+    public SerializableSqlCountNAggregateFunction(IScalarEvaluatorFactory[] args, IEvaluatorContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
+    }
+
+    @Override
+    protected void processNull(byte[] state, int start) {
+    }
+
+    @Override
+    protected long processValue(ATypeTag typeTag, long cnt) {
+        if (ATypeHierarchy.getTypeDomain(typeTag) == ATypeHierarchy.Domain.NUMERIC) {
+            return cnt + 1;
+        }
+        return cnt;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractCountAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractCountAggregateFunction.java
index 5756f00..1296d82 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractCountAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractCountAggregateFunction.java
@@ -23,6 +23,7 @@
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.om.base.AInt64;
 import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.EnumDeserializer;
@@ -45,9 +46,13 @@
     @SuppressWarnings("unchecked")
     private ISerializerDeserializer<AInt64> int64Serde =
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
+    @SuppressWarnings("unchecked")
+    private ISerializerDeserializer<ANull> nullSerde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
     private IPointable inputVal = new VoidPointable();
     private IScalarEvaluator eval;
     protected long cnt;
+    protected boolean nullRes;
 
     private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
 
@@ -60,6 +65,7 @@
     @Override
     public void init() throws HyracksDataException {
         cnt = 0;
+        nullRes = false;
     }
 
     @Override
@@ -71,7 +77,7 @@
         if (typeTag == ATypeTag.NULL || typeTag == ATypeTag.MISSING) {
             processNull();
         } else if (typeTag != ATypeTag.SYSTEM_NULL) {
-            cnt++;
+            processValue(typeTag);
         }
     }
 
@@ -79,8 +85,12 @@
     public void finish(IPointable resultPointable) throws HyracksDataException {
         resultStorage.reset();
         try {
-            result.setValue(cnt);
-            int64Serde.serialize(result, resultStorage.getDataOutput());
+            if (nullRes) {
+                nullSerde.serialize(ANull.NULL, resultStorage.getDataOutput());
+            } else {
+                result.setValue(cnt);
+                int64Serde.serialize(result, resultStorage.getDataOutput());
+            }
         } catch (IOException e) {
             throw HyracksDataException.create(e);
         }
@@ -93,4 +103,8 @@
     }
 
     protected abstract void processNull();
+
+    protected void processValue(ATypeTag tag) {
+        cnt++;
+    }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountNAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountNAggregateDescriptor.java
new file mode 100644
index 0000000..97cd1ad
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountNAggregateDescriptor.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Counts the number of numeric items. Returns NULL if MISSING or NULL is encountered.
+ */
+public class CountNAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = CountNAggregateDescriptor::new;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.COUNTN;
+    }
+
+    @Override
+    public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IAggregateEvaluator createAggregateEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+                return new CountNAggregateFunction(args, ctx, sourceLoc);
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountNAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountNAggregateFunction.java
new file mode 100644
index 0000000..c2bcb5f
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountNAggregateFunction.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.std;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+/**
+ * COUNTN returns the number of numeric items in the given list. Returns NULL if MISSING or NULL is encountered.
+ */
+public class CountNAggregateFunction extends AbstractCountAggregateFunction {
+
+    public CountNAggregateFunction(IScalarEvaluatorFactory[] args, IEvaluatorContext context, SourceLocation sourceLoc)
+            throws HyracksDataException {
+        super(args, context, sourceLoc);
+    }
+
+    @Override
+    protected void processNull() {
+        nullRes = true;
+    }
+
+    @Override
+    protected void processValue(ATypeTag tag) {
+        if (ATypeHierarchy.getTypeDomain(tag) == ATypeHierarchy.Domain.NUMERIC) {
+            cnt++;
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountNAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountNAggregateDescriptor.java
new file mode 100644
index 0000000..d68d0bb
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountNAggregateDescriptor.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Counts the number of numeric items. NULLs and MISSINGs are ignored.
+ */
+public class SqlCountNAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = SqlCountNAggregateDescriptor::new;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.SQL_COUNTN;
+    }
+
+    @Override
+    public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IAggregateEvaluatorFactory() {
+
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IAggregateEvaluator createAggregateEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+                return new SqlCountNAggregateFunction(args, ctx, sourceLoc);
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountNAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountNAggregateFunction.java
new file mode 100644
index 0000000..db677ab
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountNAggregateFunction.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.std;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+/**
+ * COUNTN returns the number of numeric items in the given list. NULLs and MISSINGs are ignored.
+ */
+public class SqlCountNAggregateFunction extends AbstractCountAggregateFunction {
+
+    public SqlCountNAggregateFunction(IScalarEvaluatorFactory[] args, IEvaluatorContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
+    }
+
+    @Override
+    protected void processNull() {
+    }
+
+    @Override
+    protected void processValue(ATypeTag tag) {
+        if (ATypeHierarchy.getTypeDomain(tag) == ATypeHierarchy.Domain.NUMERIC) {
+            cnt++;
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/CreateMBREvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/CreateMBREvalFactory.java
index 4f67fba..5615e14 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/CreateMBREvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/CreateMBREvalFactory.java
@@ -45,8 +45,8 @@
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-
-import com.esri.core.geometry.Envelope;
+import org.locationtech.jts.geom.Envelope;
+import org.locationtech.jts.geom.Geometry;
 
 public class CreateMBREvalFactory implements IScalarEvaluatorFactory {
 
@@ -257,25 +257,25 @@
                                 }
                                 break;
                             case GEOMETRY:
-                                Envelope record = new Envelope();
-                                AGeometrySerializerDeserializer.getAGeometryObject(data0, startOffset0 + 1)
-                                        .getGeometry().getEsriGeometry().queryEnvelope(record);
+                                Geometry geometry = AGeometrySerializerDeserializer
+                                        .getAGeometryObject(data0, startOffset0 + 1).getGeometry();
+                                Envelope envelope = geometry.getEnvelopeInternal();
                                 switch (coordinate) {
                                     case 0:
-                                        value = record.getXMin();
+                                        value = envelope.getMinX();
                                         break;
                                     case 1:
-                                        value = record.getYMin();
+                                        value = envelope.getMinY();
                                         break;
                                     case 2:
-                                        value = record.getXMax();
+                                        value = envelope.getMaxX();
                                         break;
                                     case 3:
-                                        value = record.getYMax();
+                                        value = envelope.getMaxY();
                                         break;
                                     default:
-                                        throw new NotImplementedException(
-                                                coordinate + "is not a valid coordinate option");
+                                        throw new IllegalArgumentException(
+                                                coordinate + " is not a valid coordinate option");
                                 }
                                 break;
                             case CIRCLE:
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLikeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLikeDescriptor.java
index 1c75aa9..6f595b1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLikeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLikeDescriptor.java
@@ -112,6 +112,7 @@
 
         @Override
         public String toRegExpPatternString(String pattern) throws HyracksDataException {
+            // note: similar logic is applied in OperatorExpressionVisitor
             tempStringBuilder.setLength(0);
             for (int i = 0, length = pattern.length(); i < length; i++) {
                 char c = pattern.charAt(i);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
index 86a0790..d682d7a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
@@ -37,6 +37,8 @@
 import org.apache.asterix.runtime.aggregates.scalar.ScalarAvgDistinctAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.scalar.ScalarCountAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.scalar.ScalarCountDistinctAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.scalar.ScalarCountNAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.scalar.ScalarCountNDistinctAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.scalar.ScalarFirstElementAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.scalar.ScalarKurtosisAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.scalar.ScalarKurtosisDistinctAggregateDescriptor;
@@ -50,6 +52,8 @@
 import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlAvgDistinctAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlCountAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlCountDistinctAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlCountNAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlCountNDistinctAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlKurtosisAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlKurtosisDistinctAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlMaxAggregateDescriptor;
@@ -83,6 +87,7 @@
 import org.apache.asterix.runtime.aggregates.scalar.ScalarVarPopDistinctAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableAvgAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableCountAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.serializable.std.SerializableCountNAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableGlobalAvgAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableGlobalKurtosisAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableGlobalSkewnessAggregateDescriptor;
@@ -135,6 +140,7 @@
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableSkewnessAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableSqlAvgAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableSqlCountAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.serializable.std.SerializableSqlCountNAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableSqlKurtosisAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableSqlSkewnessAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableSqlStddevAggregateDescriptor;
@@ -149,6 +155,7 @@
 import org.apache.asterix.runtime.aggregates.serializable.std.SerializableVarPopAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.AvgAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.CountAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.CountNAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.GlobalAvgAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.GlobalKurtosisAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.GlobalMaxAggregateDescriptor;
@@ -226,6 +233,7 @@
 import org.apache.asterix.runtime.aggregates.std.SkewnessAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.SqlAvgAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.SqlCountAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.SqlCountNAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.SqlKurtosisAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.SqlMaxAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.SqlMedianAggregateDescriptor;
@@ -698,6 +706,7 @@
         // aggregate functions
         fc.add(ListifyAggregateDescriptor.FACTORY);
         fc.add(CountAggregateDescriptor.FACTORY);
+        fc.add(CountNAggregateDescriptor.FACTORY);
         fc.add(AvgAggregateDescriptor.FACTORY);
         fc.add(LocalAvgAggregateDescriptor.FACTORY);
         fc.add(IntermediateAvgAggregateDescriptor.FACTORY);
@@ -753,6 +762,7 @@
 
         // serializable aggregates
         fc.add(SerializableCountAggregateDescriptor.FACTORY);
+        fc.add(SerializableCountNAggregateDescriptor.FACTORY);
         fc.add(SerializableAvgAggregateDescriptor.FACTORY);
         fc.add(SerializableLocalAvgAggregateDescriptor.FACTORY);
         fc.add(SerializableIntermediateAvgAggregateDescriptor.FACTORY);
@@ -790,7 +800,9 @@
         fc.add(ScalarArrayAggAggregateDescriptor.FACTORY);
         fc.add(ScalarArrayAggDistinctAggregateDescriptor.FACTORY);
         fc.add(ScalarCountAggregateDescriptor.FACTORY);
+        fc.add(ScalarCountNAggregateDescriptor.FACTORY);
         fc.add(ScalarCountDistinctAggregateDescriptor.FACTORY);
+        fc.add(ScalarCountNDistinctAggregateDescriptor.FACTORY);
         fc.add(ScalarAvgAggregateDescriptor.FACTORY);
         fc.add(ScalarAvgDistinctAggregateDescriptor.FACTORY);
         fc.add(ScalarSumAggregateDescriptor.FACTORY);
@@ -816,6 +828,7 @@
 
         // SQL aggregates
         fc.add(SqlCountAggregateDescriptor.FACTORY);
+        fc.add(SqlCountNAggregateDescriptor.FACTORY);
         fc.add(SqlAvgAggregateDescriptor.FACTORY);
         fc.add(LocalSqlAvgAggregateDescriptor.FACTORY);
         fc.add(IntermediateSqlAvgAggregateDescriptor.FACTORY);
@@ -867,6 +880,7 @@
 
         // SQL serializable aggregates
         fc.add(SerializableSqlCountAggregateDescriptor.FACTORY);
+        fc.add(SerializableSqlCountNAggregateDescriptor.FACTORY);
         fc.add(SerializableSqlAvgAggregateDescriptor.FACTORY);
         fc.add(SerializableLocalSqlAvgAggregateDescriptor.FACTORY);
         fc.add(SerializableIntermediateSqlAvgAggregateDescriptor.FACTORY);
@@ -903,6 +917,8 @@
         // SQL scalar aggregates
         fc.add(ScalarSqlCountAggregateDescriptor.FACTORY);
         fc.add(ScalarSqlCountDistinctAggregateDescriptor.FACTORY);
+        fc.add(ScalarSqlCountNDistinctAggregateDescriptor.FACTORY);
+        fc.add(ScalarSqlCountNAggregateDescriptor.FACTORY);
         fc.add(ScalarSqlAvgAggregateDescriptor.FACTORY);
         fc.add(ScalarSqlAvgDistinctAggregateDescriptor.FACTORY);
         fc.add(ScalarSqlSumAggregateDescriptor.FACTORY);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/DynamicPathResolver.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/DynamicPathResolver.java
index c6d1dbe..e3ccc33 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/DynamicPathResolver.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/DynamicPathResolver.java
@@ -32,7 +32,7 @@
 import org.apache.hyracks.util.string.UTF8CharBuffer;
 import org.apache.hyracks.util.string.UTF8StringUtil;
 
-final class DynamicPathResolver extends AbstractPathResolver {
+public final class DynamicPathResolver extends AbstractPathResolver {
     private final IScalarEvaluator pathEval;
     private final IWarningCollector warningCollector;
     private final StringBuilder dirStringBuilder;
@@ -40,7 +40,7 @@
     private final UTF8CharBuffer charBuffer;
     private final SourceLocation pathSourceLocation;
 
-    DynamicPathResolver(String fileExtension, char fileSeparator, int partition, IScalarEvaluator pathEval,
+    public DynamicPathResolver(String fileExtension, char fileSeparator, int partition, IScalarEvaluator pathEval,
             IWarningCollector warningCollector, SourceLocation pathSourceLocation) {
         super(fileExtension, fileSeparator, partition);
         this.pathEval = pathEval;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalFileWriter.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalFileWriter.java
index 95dc962..e8bae00 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalFileWriter.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalFileWriter.java
@@ -23,7 +23,7 @@
 import org.apache.hyracks.data.std.api.IValueReference;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-final class ExternalFileWriter implements IExternalWriter {
+public final class ExternalFileWriter implements IExternalWriter {
     static final String UNRESOLVABLE_PATH = "UNRESOLVABLE_PATH";
     private final IPathResolver pathResolver;
     private final IExternalFileWriter writer;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalFileWriterFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalFileWriterFactory.java
index 5981584..f85a8ab 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalFileWriterFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalFileWriterFactory.java
@@ -18,52 +18,29 @@
  */
 package org.apache.asterix.runtime.writer;
 
-import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
-import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.algebricks.runtime.evaluators.EvaluatorContext;
 import org.apache.hyracks.algebricks.runtime.writers.IExternalWriter;
 import org.apache.hyracks.algebricks.runtime.writers.IExternalWriterFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.exceptions.IWarningCollector;
-import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class ExternalFileWriterFactory implements IExternalWriterFactory {
     private static final long serialVersionUID = 1412969574113419638L;
     private final IExternalFileWriterFactory writerFactory;
     private final IExternalPrinterFactory printerFactory;
-    private final String fileExtension;
     private final int maxResult;
-    private final IScalarEvaluatorFactory pathEvalFactory;
-    private final String staticPath;
-    private final SourceLocation pathSourceLocation;
+    private final IPathResolverFactory pathResolverFactory;
 
     public ExternalFileWriterFactory(IExternalFileWriterFactory writerFactory, IExternalPrinterFactory printerFactory,
-            String fileExtension, int maxResult, IScalarEvaluatorFactory pathEvalFactory, String staticPath,
-            SourceLocation pathSourceLocation) {
+            IPathResolverFactory pathResolverFactory, int maxResult) {
         this.writerFactory = writerFactory;
         this.printerFactory = printerFactory;
-        this.fileExtension = fileExtension;
+        this.pathResolverFactory = pathResolverFactory;
         this.maxResult = maxResult;
-        this.pathEvalFactory = pathEvalFactory;
-        this.staticPath = staticPath;
-        this.pathSourceLocation = pathSourceLocation;
     }
 
     @Override
     public IExternalWriter createWriter(IHyracksTaskContext context) throws HyracksDataException {
-        int partition = context.getTaskAttemptId().getTaskId().getPartition();
-        char fileSeparator = writerFactory.getSeparator();
-        IPathResolver resolver;
-        if (staticPath == null) {
-            EvaluatorContext evaluatorContext = new EvaluatorContext(context);
-            IScalarEvaluator pathEval = pathEvalFactory.createScalarEvaluator(evaluatorContext);
-            IWarningCollector warningCollector = context.getWarningCollector();
-            resolver = new DynamicPathResolver(fileExtension, fileSeparator, partition, pathEval, warningCollector,
-                    pathSourceLocation);
-        } else {
-            resolver = new StaticPathResolver(fileExtension, fileSeparator, partition, staticPath);
-        }
+        IPathResolver resolver = pathResolverFactory.createResolver(context);
         IExternalFileWriter writer = writerFactory.createWriter(context, printerFactory);
         return new ExternalFileWriter(resolver, writer, maxResult);
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalWriterFactoryValidator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalWriterFactoryValidator.java
index 4a75db6..2aeca4f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalWriterFactoryValidator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalWriterFactoryValidator.java
@@ -18,11 +18,12 @@
  */
 package org.apache.asterix.runtime.writer;
 
+import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 
 public interface IExternalWriterFactoryValidator {
     /**
      * Perform the necessary validation to ensure the writer has the proper permissions
      */
-    void validate() throws AlgebricksException;
+    void validate(IApplicationContext appCtx) throws AlgebricksException;
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IPathResolver.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IPathResolver.java
index 35b4ddd..c4a3b04 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IPathResolver.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IPathResolver.java
@@ -24,7 +24,7 @@
 /**
  * Path resolver which generates paths for the written files
  */
-interface IPathResolver {
+public interface IPathResolver {
 
     /**
      * Extract the partitioning values from the provided tuple and generates the file path
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IPathResolverFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IPathResolverFactory.java
new file mode 100644
index 0000000..18a8799
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IPathResolverFactory.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.writer;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface IPathResolverFactory extends Serializable {
+
+    IPathResolver createResolver(IHyracksTaskContext ctx) throws HyracksDataException;
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/PathResolverFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/PathResolverFactory.java
new file mode 100644
index 0000000..8707cae
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/PathResolverFactory.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.writer;
+
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.evaluators.EvaluatorContext;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public class PathResolverFactory implements IPathResolverFactory {
+    private static final long serialVersionUID = 8971234908711238L;
+    private final IExternalFileWriterFactory writerFactory;
+    private final String fileExtension;
+    private final IScalarEvaluatorFactory pathEvalFactory;
+    private final String staticPath;
+    private final SourceLocation pathSourceLocation;
+
+    public PathResolverFactory(IExternalFileWriterFactory writerFactory, String fileExtension,
+            IScalarEvaluatorFactory pathEvalFactory, String staticPath, SourceLocation pathSourceLocation) {
+        this.writerFactory = writerFactory;
+        this.fileExtension = fileExtension;
+        this.pathEvalFactory = pathEvalFactory;
+        this.pathSourceLocation = pathSourceLocation;
+        this.staticPath = staticPath;
+    }
+
+    @Override
+    public IPathResolver createResolver(IHyracksTaskContext context) throws HyracksDataException {
+        int partition = context.getTaskAttemptId().getTaskId().getPartition();
+        char fileSeparator = writerFactory.getSeparator();
+        IPathResolver resolver;
+        if (staticPath == null) {
+            EvaluatorContext evaluatorContext = new EvaluatorContext(context);
+            IScalarEvaluator pathEval = pathEvalFactory.createScalarEvaluator(evaluatorContext);
+            IWarningCollector warningCollector = context.getWarningCollector();
+            resolver = new DynamicPathResolver(fileExtension, fileSeparator, partition, pathEval, warningCollector,
+                    pathSourceLocation);
+        } else {
+            resolver = new StaticPathResolver(fileExtension, fileSeparator, partition, staticPath);
+        }
+        return resolver;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/StaticPathResolver.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/StaticPathResolver.java
index 52943ed..c5fa0dd 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/StaticPathResolver.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/StaticPathResolver.java
@@ -20,10 +20,10 @@
 
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-final class StaticPathResolver extends AbstractPathResolver {
+public final class StaticPathResolver extends AbstractPathResolver {
     private final String directoryPath;
 
-    StaticPathResolver(String fileExtension, char fileSeparator, int partition, String directoryPath) {
+    public StaticPathResolver(String fileExtension, char fileSeparator, int partition, String directoryPath) {
         super(fileExtension, fileSeparator, partition);
 
         if (!directoryPath.isEmpty() && directoryPath.charAt(directoryPath.length() - 1) != fileSeparator) {
diff --git a/asterixdb/asterix-server/pom.xml b/asterixdb/asterix-server/pom.xml
index 83ee04d..579429b 100644
--- a/asterixdb/asterix-server/pom.xml
+++ b/asterixdb/asterix-server/pom.xml
@@ -621,6 +621,22 @@
               <contentFile>protobuf_2.5.0_LICENSE.txt</contentFile>
             </license>
             <license>
+              <displayName>Eclipse Distribution License - v 1.0</displayName>
+              <url>https://www.eclipse.org/org/documents/edl-v10.php</url>
+              <aliasUrls>
+                <aliasUrl>https://github.com/locationtech/jts/blob/master/LICENSE_EDLv1.txt</aliasUrl>
+              </aliasUrls>
+              <metric>1050</metric>
+            </license>
+            <license>
+              <displayName>Eclipse Public License, Version 2.0</displayName>
+              <url>https://www.eclipse.org/legal/epl-2.0/</url>
+              <aliasUrls>
+                <aliasUrl>https://github.com/locationtech/jts/blob/master/LICENSE_EPLv2.txt</aliasUrl>
+              </aliasUrls>
+              <metric>1100</metric>
+            </license>
+            <license>
               <displayName>a BSD 3-clause license</displayName>
               <url>https://github.com/codehaus/paranamer-git/blob/paranamer-2.3/LICENSE.txt</url>
               <contentFile>paranamer-2.3_LICENSE.txt</contentFile>
diff --git a/asterixdb/asterix-server/src/test/resources/NCServiceExecutionIT/cc.conf b/asterixdb/asterix-server/src/test/resources/NCServiceExecutionIT/cc.conf
index 1c6d574..5b159ea 100644
--- a/asterixdb/asterix-server/src/test/resources/NCServiceExecutionIT/cc.conf
+++ b/asterixdb/asterix-server/src/test/resources/NCServiceExecutionIT/cc.conf
@@ -47,4 +47,5 @@
 
 [common]
 log.dir = ../asterix-server/target/NCServiceExecutionIT
-log.level = INFO
\ No newline at end of file
+log.level = INFO
+compiler.ordered.fields=false
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/NcLifecycleIT/cc.conf b/asterixdb/asterix-server/src/test/resources/NcLifecycleIT/cc.conf
index a91622b..a5a437a 100644
--- a/asterixdb/asterix-server/src/test/resources/NcLifecycleIT/cc.conf
+++ b/asterixdb/asterix-server/src/test/resources/NcLifecycleIT/cc.conf
@@ -47,3 +47,4 @@
 [common]
 log.level = INFO
 log.dir=../asterix-server/target/NcLifecycleIT
+compiler.ordered.fields=false
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/ReplicationIT/cc.conf b/asterixdb/asterix-server/src/test/resources/ReplicationIT/cc.conf
index b88cbd4..bef0a3c 100644
--- a/asterixdb/asterix-server/src/test/resources/ReplicationIT/cc.conf
+++ b/asterixdb/asterix-server/src/test/resources/ReplicationIT/cc.conf
@@ -51,3 +51,4 @@
 replication.enabled=true
 replication.strategy=all
 replication.factor=2
+compiler.ordered.fields=false
\ No newline at end of file
diff --git a/asterixdb/pom.xml b/asterixdb/pom.xml
index c0f9792..4b91cfe 100644
--- a/asterixdb/pom.xml
+++ b/asterixdb/pom.xml
@@ -96,7 +96,7 @@
     <log4j.version>2.22.1</log4j.version>
     <awsjavasdk.version>2.29.27</awsjavasdk.version>
     <awsjavasdk.crt.version>0.29.10</awsjavasdk.crt.version>
-    <parquet.version>1.14.1</parquet.version>
+    <parquet.version>1.14.3</parquet.version>
     <hadoop-awsjavasdk.version>1.12.779</hadoop-awsjavasdk.version>
     <azureblobjavasdk.version>12.25.1</azureblobjavasdk.version>
     <azurecommonjavasdk.version>12.24.1</azurecommonjavasdk.version>
@@ -674,7 +674,7 @@
             <dependency>
               <groupId>com.github.javaparser</groupId>
               <artifactId>javaparser-core</artifactId>
-              <version>3.25.5</version>
+              <version>3.26.3</version>
             </dependency>
           </dependencies>
         </plugin>
@@ -750,14 +750,6 @@
     </profile>
     <profile>
       <id>azurite-tests</id>
-      <activation>
-        <os>
-          <family>unix</family>
-        </os>
-        <property>
-          <name>!skipTests</name>
-        </property>
-      </activation>
       <properties>
         <azurite.npm.install.stage>process-classes</azurite.npm.install.stage>
         <azurite.install.stage>generate-test-resources</azurite.install.stage>
@@ -1568,9 +1560,9 @@
         <version>3.8.4</version>
       </dependency>
       <dependency>
-        <groupId>com.esri.geometry</groupId>
-        <artifactId>esri-geometry-api</artifactId>
-        <version>2.0.0</version>
+        <groupId>org.locationtech.jts</groupId>
+        <artifactId>jts-core</artifactId>
+        <version>1.19.0</version>
       </dependency>
       <dependency>
         <groupId>org.reflections</groupId>
diff --git a/asterixdb/src/main/licenses/content/com.microsoft.azure--msal4j--1.16.1_MIT_License.txt b/asterixdb/src/main/licenses/content/com.microsoft.azure--msal4j--1.16.1_MIT_License.txt
new file mode 100644
index 0000000..d1ca00f
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/com.microsoft.azure--msal4j--1.16.1_MIT_License.txt
@@ -0,0 +1,21 @@
+    MIT License
+
+    Copyright (c) Microsoft Corporation. All rights reserved.
+
+    Permission is hereby granted, free of charge, to any person obtaining a copy
+    of this software and associated documentation files (the "Software"), to deal
+    in the Software without restriction, including without limitation the rights
+    to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+    copies of the Software, and to permit persons to whom the Software is
+    furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be included in all
+    copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+    OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+    SOFTWARE
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/com.microsoft.azure--msal4j-persistence-extension--1.3.0_MIT_License.txt b/asterixdb/src/main/licenses/content/com.microsoft.azure--msal4j-persistence-extension--1.3.0_MIT_License.txt
new file mode 100644
index 0000000..d1ca00f
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/com.microsoft.azure--msal4j-persistence-extension--1.3.0_MIT_License.txt
@@ -0,0 +1,21 @@
+    MIT License
+
+    Copyright (c) Microsoft Corporation. All rights reserved.
+
+    Permission is hereby granted, free of charge, to any person obtaining a copy
+    of this software and associated documentation files (the "Software"), to deal
+    in the Software without restriction, including without limitation the rights
+    to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+    copies of the Software, and to permit persons to whom the Software is
+    furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be included in all
+    copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+    OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+    SOFTWARE
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/github.com_locationtech_jts_blob_master_LICENSE_EDLv1.txt b/asterixdb/src/main/licenses/content/github.com_locationtech_jts_blob_master_LICENSE_EDLv1.txt
new file mode 100644
index 0000000..1071fed
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/github.com_locationtech_jts_blob_master_LICENSE_EDLv1.txt
@@ -0,0 +1,30 @@
+Eclipse Distribution License - v 1.0
+
+Copyright (c) 2007, Eclipse Foundation, Inc. and its licensors.
+
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+  Redistributions of source code must retain the above copyright notice, this
+  list of conditions and the following disclaimer.
+
+  Redistributions in binary form must reproduce the above copyright notice,
+  this list of conditions and the following disclaimer in the documentation
+  and/or other materials provided with the distribution.
+
+  Neither the name of the Eclipse Foundation, Inc. nor the names of its
+  contributors may be used to endorse or promote products derived from this
+  software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/opensource.org_licenses_BSD-2-Clause.txt b/asterixdb/src/main/licenses/content/opensource.org_licenses_BSD-2-Clause.txt
new file mode 100644
index 0000000..226190a
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/opensource.org_licenses_BSD-2-Clause.txt
@@ -0,0 +1,24 @@
+BSD 2-Clause License
+
+Copyright (c) 2024, Couchbase, Inc.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+1. Redistributions of source code must retain the above copyright notice, this
+   list of conditions and the following disclaimer.
+
+2. Redistributions in binary form must reproduce the above copyright notice,
+   this list of conditions and the following disclaimer in the documentation
+   and/or other materials provided with the distribution.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE
+FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
+SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
+CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
+OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/templates/source_licenses.ftl b/asterixdb/src/main/licenses/templates/source_licenses.ftl
index 97b032b..95b6f3a 100644
--- a/asterixdb/src/main/licenses/templates/source_licenses.ftl
+++ b/asterixdb/src/main/licenses/templates/source_licenses.ftl
@@ -133,4 +133,196 @@
 </#if>
 <#if !asterixDashboardSkip!false>
     <#include "../../../../asterix-dashboard/src/main/licenses/dashboard-source-license.ftl">
-</#if>
\ No newline at end of file
+</#if>
+<@license component="AsterixDB OM" files=[
+                    "org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/JtsModule.java",
+                    "org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/MultiLineStringParser.java",
+                    "org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/PointParser.java",
+                    "org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/MultiPointParser.java",
+                    "org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/GeometryParser.java",
+                    "org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/MultiPolygonParser.java",
+                    "org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/LineStringParser.java",
+                    "org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/BaseParser.java",
+                    "org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/GeometryCollectionParser.java",
+                    "org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/GenericGeometryParser.java",
+                    "org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/parsers/PolygonParser.java",
+                    "org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/GeoFunctionUtils.java",
+                    "org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/GeoJsonConstants.java",
+                    "org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/serde/GeometryDeserializer.java",
+                    "org/apache/asterix/dataflow/data/nontagged/serde/jacksonjts/serde/GeometrySerializer.java"
+                ]>
+    Classes are modified to support the org.locationtech.jts package instead of com.vividsolutions.jts
+    Copyright BeDataDriven
+
+    Apache License
+    Version 2.0, January 2004
+    http://www.apache.org/licenses/
+    TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+    1. Definitions.
+    "License" shall mean the terms and conditions for use, reproduction,
+    and distribution as defined by Sections 1 through 9 of this document.
+    "Licensor" shall mean the copyright owner or entity authorized by
+    the copyright owner that is granting the License.
+    "Legal Entity" shall mean the union of the acting entity and all
+    other entities that control, are controlled by, or are under common
+    control with that entity. For the purposes of this definition,
+    "control" means (i) the power, direct or indirect, to cause the
+    direction or management of such entity, whether by contract or
+    otherwise, or (ii) ownership of fifty percent (50%) or more of the
+    outstanding shares, or (iii) beneficial ownership of such entity.
+    "You" (or "Your") shall mean an individual or Legal Entity
+    exercising permissions granted by this License.
+    "Source" form shall mean the preferred form for making modifications,
+    including but not limited to software source code, documentation
+    source, and configuration files.
+    "Object" form shall mean any form resulting from mechanical
+    transformation or translation of a Source form, including but
+    not limited to compiled object code, generated documentation,
+    and conversions to other media types.
+    "Work" shall mean the work of authorship, whether in Source or
+    Object form, made available under the License, as indicated by a
+    copyright notice that is included in or attached to the work
+    (an example is provided in the Appendix below).
+    "Derivative Works" shall mean any work, whether in Source or Object
+    form, that is based on (or derived from) the Work and for which the
+    editorial revisions, annotations, elaborations, or other modifications
+    represent, as a whole, an original work of authorship. For the purposes
+    of this License, Derivative Works shall not include works that remain
+    separable from, or merely link (or bind by name) to the interfaces of,
+    the Work and Derivative Works thereof.
+    "Contribution" shall mean any work of authorship, including
+    the original version of the Work and any modifications or additions
+    to that Work or Derivative Works thereof, that is intentionally
+    submitted to Licensor for inclusion in the Work by the copyright owner
+    or by an individual or Legal Entity authorized to submit on behalf of
+    the copyright owner. For the purposes of this definition, "submitted"
+    means any form of electronic, verbal, or written communication sent
+    to the Licensor or its representatives, including but not limited to
+    communication on electronic mailing lists, source code control systems,
+    and issue tracking systems that are managed by, or on behalf of, the
+    Licensor for the purpose of discussing and improving the Work, but
+    excluding communication that is conspicuously marked or otherwise
+    designated in writing by the copyright owner as "Not a Contribution."
+    "Contributor" shall mean Licensor and any individual or Legal Entity
+    on behalf of whom a Contribution has been received by Licensor and
+    subsequently incorporated within the Work.
+    2. Grant of Copyright License. Subject to the terms and conditions of
+    this License, each Contributor hereby grants to You a perpetual,
+    worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+    copyright license to reproduce, prepare Derivative Works of,
+    publicly display, publicly perform, sublicense, and distribute the
+    Work and such Derivative Works in Source or Object form.
+    3. Grant of Patent License. Subject to the terms and conditions of
+    this License, each Contributor hereby grants to You a perpetual,
+    worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+    (except as stated in this section) patent license to make, have made,
+    use, offer to sell, sell, import, and otherwise transfer the Work,
+    where such license applies only to those patent claims licensable
+    by such Contributor that are necessarily infringed by their
+    Contribution(s) alone or by combination of their Contribution(s)
+    with the Work to which such Contribution(s) was submitted. If You
+    institute patent litigation against any entity (including a
+    cross-claim or counterclaim in a lawsuit) alleging that the Work
+    or a Contribution incorporated within the Work constitutes direct
+    or contributory patent infringement, then any patent licenses
+    granted to You under this License for that Work shall terminate
+    as of the date such litigation is filed.
+    4. Redistribution. You may reproduce and distribute copies of the
+    Work or Derivative Works thereof in any medium, with or without
+    modifications, and in Source or Object form, provided that You
+    meet the following conditions:
+    (a) You must give any other recipients of the Work or
+    Derivative Works a copy of this License; and
+    (b) You must cause any modified files to carry prominent notices
+    stating that You changed the files; and
+    (c) You must retain, in the Source form of any Derivative Works
+    that You distribute, all copyright, patent, trademark, and
+    attribution notices from the Source form of the Work,
+    excluding those notices that do not pertain to any part of
+    the Derivative Works; and
+    (d) If the Work includes a "NOTICE" text file as part of its
+    distribution, then any Derivative Works that You distribute must
+    include a readable copy of the attribution notices contained
+    within such NOTICE file, excluding those notices that do not
+    pertain to any part of the Derivative Works, in at least one
+    of the following places: within a NOTICE text file distributed
+    as part of the Derivative Works; within the Source form or
+    documentation, if provided along with the Derivative Works; or,
+    within a display generated by the Derivative Works, if and
+    wherever such third-party notices normally appear. The contents
+    of the NOTICE file are for informational purposes only and
+    do not modify the License. You may add Your own attribution
+    notices within Derivative Works that You distribute, alongside
+    or as an addendum to the NOTICE text from the Work, provided
+    that such additional attribution notices cannot be construed
+    as modifying the License.
+    You may add Your own copyright statement to Your modifications and
+    may provide additional or different license terms and conditions
+    for use, reproduction, or distribution of Your modifications, or
+    for any such Derivative Works as a whole, provided Your use,
+    reproduction, and distribution of the Work otherwise complies with
+    the conditions stated in this License.
+    5. Submission of Contributions. Unless You explicitly state otherwise,
+    any Contribution intentionally submitted for inclusion in the Work
+    by You to the Licensor shall be under the terms and conditions of
+    this License, without any additional terms or conditions.
+    Notwithstanding the above, nothing herein shall supersede or modify
+    the terms of any separate license agreement you may have executed
+    with Licensor regarding such Contributions.
+    6. Trademarks. This License does not grant permission to use the trade
+    names, trademarks, service marks, or product names of the Licensor,
+    except as required for reasonable and customary use in describing the
+    origin of the Work and reproducing the content of the NOTICE file.
+    7. Disclaimer of Warranty. Unless required by applicable law or
+    agreed to in writing, Licensor provides the Work (and each
+    Contributor provides its Contributions) on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+    implied, including, without limitation, any warranties or conditions
+    of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+    PARTICULAR PURPOSE. You are solely responsible for determining the
+    appropriateness of using or redistributing the Work and assume any
+    risks associated with Your exercise of permissions under this License.
+    8. Limitation of Liability. In no event and under no legal theory,
+    whether in tort (including negligence), contract, or otherwise,
+    unless required by applicable law (such as deliberate and grossly
+    negligent acts) or agreed to in writing, shall any Contributor be
+    liable to You for damages, including any direct, indirect, special,
+    incidental, or consequential damages of any character arising as a
+    result of this License or out of the use or inability to use the
+    Work (including but not limited to damages for loss of goodwill,
+    work stoppage, computer failure or malfunction, or any and all
+    other commercial damages or losses), even if such Contributor
+    has been advised of the possibility of such damages.
+    9. Accepting Warranty or Additional Liability. While redistributing
+    the Work or Derivative Works thereof, You may choose to offer,
+    and charge a fee for, acceptance of support, warranty, indemnity,
+    or other liability obligations and/or rights consistent with this
+    License. However, in accepting such obligations, You may act only
+    on Your own behalf and on Your sole responsibility, not on behalf
+    of any other Contributor, and only if You agree to indemnify,
+    defend, and hold each Contributor harmless for any liability
+    incurred by, or claims asserted against, such Contributor by reason
+    of your accepting any such warranty or additional liability.
+    END OF TERMS AND CONDITIONS
+    APPENDIX: How to apply the Apache License to your work.
+    To apply the Apache License to your work, attach the following
+    boilerplate notice, with the fields enclosed by brackets "[]"
+    replaced with your own identifying information. (Don't include
+    the brackets!)  The text should be enclosed in the appropriate
+    comment syntax for the file format. We also recommend that a
+    file or class name and description of purpose be included on the
+    same "printed page" as the copyright notice for easier
+    identification within third-party archives.
+    Copyright [yyyy] [name of copyright owner]
+    Licensed under the Apache License, Version 2.0 (the "License");
+    you may not use this file except in compliance with the License.
+    You may obtain a copy of the License at
+    http://www.apache.org/licenses/LICENSE-2.0
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+    For more details on the original project and licensing, please visit
+    https://github.com/bedatadriven/jackson-datatype-jts.
+</@license>
\ No newline at end of file
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/AbstractAssignPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/AbstractAssignPOperator.java
index 4466a25..319dd3b 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/AbstractAssignPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/AbstractAssignPOperator.java
@@ -66,12 +66,29 @@
         AssignOperator assign = (AssignOperator) op;
         List<LogicalVariable> variables = assign.getVariables();
         int[] outColumns = new int[variables.size()];
-        for (int i = 0; i < outColumns.length; i++) {
-            outColumns[i] = opSchema.findVariable(variables.get(i));
-        }
+        int[] projectionList;
 
-        // TODO push projections into the operator
-        int[] projectionList = JobGenHelper.projectAllVariables(opSchema);
+        if (assign.isProjectPushed()) {
+            for (int i = 0; i < outColumns.length; i++) {
+                outColumns[i] = inputSchemas[0].getSize() + i;
+            }
+            List<LogicalVariable> projectVars = assign.getProjectVariables();
+
+            projectionList = new int[projectVars.size()];
+            int c = 0;
+            for (LogicalVariable projectVar : projectVars) {
+                if (variables.contains(projectVar)) {
+                    projectionList[c++] = inputSchemas[0].getSize() + variables.indexOf(projectVar);
+                } else {
+                    projectionList[c++] = inputSchemas[0].findVariable(projectVar);
+                }
+            }
+        } else {
+            for (int i = 0; i < outColumns.length; i++) {
+                outColumns[i] = opSchema.findVariable(variables.get(i));
+            }
+            projectionList = JobGenHelper.projectAllVariables(opSchema);
+        }
 
         IPushRuntimeFactory runtime =
                 createRuntimeFactory(context, assign, opSchema, inputSchemas, outColumns, projectionList);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractAssignOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractAssignOperator.java
index 586b1a8..35d943f 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractAssignOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractAssignOperator.java
@@ -31,16 +31,16 @@
 /**
  * @author Nicola
  */
-public abstract class AbstractAssignOperator extends AbstractLogicalOperator {
+public abstract class AbstractAssignOperator extends AbstractProjectingOperator {
     protected final List<LogicalVariable> variables;
     protected final List<Mutable<ILogicalExpression>> expressions;
 
     public AbstractAssignOperator() {
-        this.variables = new ArrayList<LogicalVariable>();
-        this.expressions = new ArrayList<Mutable<ILogicalExpression>>();
+        this(new ArrayList<>(), new ArrayList<>());
     }
 
     public AbstractAssignOperator(List<LogicalVariable> variables, List<Mutable<ILogicalExpression>> expressions) {
+        super();
         this.variables = variables;
         this.expressions = expressions;
     }
@@ -56,6 +56,12 @@
     @Override
     public void recomputeSchema() {
         schema = new ArrayList<LogicalVariable>();
+        if (isProjectPushed()) {
+            for (LogicalVariable p : getProjectVariables()) {
+                schema.add(p);
+            }
+            return;
+        }
         schema.addAll(inputs.get(0).getValue().getSchema());
         schema.addAll(variables);
     }
@@ -75,6 +81,12 @@
         return new VariablePropagationPolicy() {
             @Override
             public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources) {
+                if (isProjectPushed()) {
+                    for (LogicalVariable p : getProjectVariables()) {
+                        target.addVariable(p);
+                    }
+                    return;
+                }
                 if (propagateInputVars) {
                     target.addAllVariables(sources[0]);
                 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractProjectingOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractProjectingOperator.java
new file mode 100644
index 0000000..8b36113
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractProjectingOperator.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+
+public abstract class AbstractProjectingOperator extends AbstractLogicalOperator {
+    private final List<LogicalVariable> projectVars;
+    private boolean projectPushed;
+
+    public AbstractProjectingOperator() {
+        projectPushed = false;
+        projectVars = new ArrayList<>();
+    }
+
+    public void pushProjectionVariables(List<LogicalVariable> vars) {
+        projectPushed = true;
+        projectVars.addAll(vars);
+    }
+
+    public List<LogicalVariable> getProjectVariables() {
+        return projectVars;
+    }
+
+    public boolean isProjectPushed() {
+        return projectPushed;
+    }
+
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractScanOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractScanOperator.java
index 64e41f5..8559bb6 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractScanOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractScanOperator.java
@@ -25,7 +25,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
 
-public abstract class AbstractScanOperator extends AbstractLogicalOperator {
+public abstract class AbstractScanOperator extends AbstractProjectingOperator {
     protected List<LogicalVariable> variables;
 
     public AbstractScanOperator(List<LogicalVariable> variables) {
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestNonMapOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestNonMapOperator.java
index 5c485d5..77bf37e 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestNonMapOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestNonMapOperator.java
@@ -25,6 +25,7 @@
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.properties.FilteredVariablePropagationPolicy;
 import org.apache.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
 
 public abstract class AbstractUnnestNonMapOperator extends AbstractUnnestOperator {
@@ -85,7 +86,25 @@
     }
 
     @Override
+    public void recomputeSchema() {
+        schema = new ArrayList<>();
+        if (isProjectPushed()) {
+            for (LogicalVariable v : getProjectVariables()) {
+                schema.add(v);
+            }
+            return;
+        }
+
+        schema.addAll(inputs.get(0).getValue().getSchema());
+        schema.addAll(variables);
+    }
+
+    @Override
     public VariablePropagationPolicy getVariablePropagationPolicy() {
+
+        if (isProjectPushed()) {
+            return new FilteredVariablePropagationPolicy(getProjectVariables());
+        }
         return new VariablePropagationPolicy() {
 
             @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
index c0f71fe..c5310b7 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
@@ -18,8 +18,6 @@
  */
 package org.apache.hyracks.algebricks.core.algebra.operators.logical;
 
-import java.util.ArrayList;
-import java.util.Collection;
 import java.util.List;
 
 import org.apache.commons.lang3.mutable.Mutable;
@@ -37,10 +35,6 @@
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
 
 public class DataSourceScanOperator extends AbstractDataSourceOperator {
-    private final List<LogicalVariable> projectVars;
-
-    private boolean projectPushed = false;
-
     private List<Mutable<ILogicalExpression>> additionalFilteringExpressions;
     private List<LogicalVariable> minFilterVars;
     private List<LogicalVariable> maxFilterVars;
@@ -61,7 +55,6 @@
             Mutable<ILogicalExpression> selectCondition, long outputLimit,
             IProjectionFiltrationInfo projectionFiltrationInfo) {
         super(variables, dataSource);
-        projectVars = new ArrayList<>();
         this.selectCondition = selectCondition;
         this.outputLimit = outputLimit;
         setProjectionFiltrationInfo(projectionFiltrationInfo);
@@ -93,19 +86,6 @@
         return false;
     }
 
-    public void addProjectVariables(Collection<LogicalVariable> vars) {
-        projectVars.addAll(vars);
-        projectPushed = true;
-    }
-
-    public List<LogicalVariable> getProjectVariables() {
-        return projectVars;
-    }
-
-    public boolean isProjectPushed() {
-        return projectPushed;
-    }
-
     @Override
     public VariablePropagationPolicy getVariablePropagationPolicy() {
         return new VariablePropagationPolicy() {
@@ -115,7 +95,7 @@
                 if (sources.length > 0) {
                     target.addAllVariables(sources[0]);
                 }
-                List<LogicalVariable> outputVariables = projectPushed ? projectVars : variables;
+                List<LogicalVariable> outputVariables = isProjectPushed() ? getProjectVariables() : variables;
                 for (LogicalVariable v : outputVariables) {
                     target.addVariable(v);
                 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java
index 2d2910f..ebb7ec6 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java
@@ -34,6 +34,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.properties.FilteredVariablePropagationPolicy;
 import org.apache.hyracks.algebricks.core.algebra.properties.TypePropagationPolicy;
 import org.apache.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
 import org.apache.hyracks.algebricks.core.algebra.typing.ITypeEnvPointer;
@@ -43,7 +44,7 @@
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
 
-public class SelectOperator extends AbstractLogicalOperator {
+public class SelectOperator extends AbstractProjectingOperator {
     private final Mutable<ILogicalExpression> condition;
     private final IAlgebricksConstantValue retainMissingAsValue;
     private LogicalVariable missingPlaceholderVar;
@@ -96,11 +97,22 @@
 
     @Override
     public void recomputeSchema() {
+        if (isProjectPushed()) {
+            schema = new ArrayList<>();
+            for (LogicalVariable v : getProjectVariables()) {
+                schema.add(v);
+            }
+            return;
+        }
+
         schema = new ArrayList<>(inputs.get(0).getValue().getSchema());
     }
 
     @Override
     public VariablePropagationPolicy getVariablePropagationPolicy() {
+        if (isProjectPushed()) {
+            return new FilteredVariablePropagationPolicy(getProjectVariables());
+        }
         return VariablePropagationPolicy.ALL;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractUnnestPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractUnnestPOperator.java
index 45360d4..199650c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractUnnestPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractUnnestPOperator.java
@@ -24,6 +24,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression.FunctionKind;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionRuntimeProvider;
@@ -64,7 +65,6 @@
             IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
             throws AlgebricksException {
         AbstractUnnestNonMapOperator unnest = (AbstractUnnestNonMapOperator) op;
-        int outCol = opSchema.findVariable(unnest.getVariable());
         ILogicalExpression unnestExpr = unnest.getExpressionRef().getValue();
         IExpressionRuntimeProvider expressionRuntimeProvider = context.getExpressionRuntimeProvider();
         boolean exit = false;
@@ -83,14 +83,40 @@
         UnnestingFunctionCallExpression agg = (UnnestingFunctionCallExpression) unnestExpr;
         IUnnestingEvaluatorFactory unnestingFactory = expressionRuntimeProvider.createUnnestingFunctionFactory(agg,
                 context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas, context);
-        int[] projectionList = JobGenHelper.projectAllVariables(opSchema);
         IUnnestingPositionWriterFactory positionWriterFactory =
                 unnest.hasPositionalVariable() ? context.getUnnestingPositionWriterFactory() : null;
         IMissingWriterFactory missingWriterFactory = leftOuter
                 ? JobGenHelper.getMissingWriterFactory(context, ((LeftOuterUnnestOperator) op).getMissingValue())
                 : null;
-        UnnestRuntimeFactory unnestRuntime = new UnnestRuntimeFactory(outCol, unnestingFactory, projectionList,
-                positionWriterFactory, leftOuter, missingWriterFactory);
+
+        int outCol;
+        int positionalCol;
+        int[] projectionList;
+
+        if (unnest.isProjectPushed()) {
+            outCol = -1;
+            positionalCol = -1;
+            projectionList = new int[unnest.getProjectVariables().size()];
+            int c = 0;
+            for (LogicalVariable projectVar : unnest.getProjectVariables()) {
+                if (projectVar.equals(unnest.getVariable())) {
+                    outCol = inputSchemas[0].getSize();
+                    projectionList[c++] = inputSchemas[0].getSize();
+                } else if (unnest.hasPositionalVariable() && projectVar.equals(unnest.getPositionalVariable())) {
+                    positionalCol = inputSchemas[0].getSize() + 1;
+                    projectionList[c++] = inputSchemas[0].getSize() + 1;
+                } else {
+                    projectionList[c++] = inputSchemas[0].findVariable(projectVar);
+                }
+            }
+        } else {
+            outCol = opSchema.findVariable(unnest.getVariable());
+            positionalCol = unnest.hasPositionalVariable() ? opSchema.findVariable(unnest.getPositionalVariable()) : -1;
+            projectionList = JobGenHelper.projectAllVariables(opSchema);
+        }
+
+        UnnestRuntimeFactory unnestRuntime = new UnnestRuntimeFactory(outCol, positionalCol, unnestingFactory,
+                projectionList, positionWriterFactory, leftOuter, missingWriterFactory);
         unnestRuntime.setSourceLocation(unnest.getSourceLocation());
         RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
         builder.contributeMicroOperator(unnest, unnestRuntime, recDesc);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java
index 4a0c85f..506457e 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java
@@ -73,10 +73,6 @@
         RunningAggregateOperator ragg = (RunningAggregateOperator) op;
         List<LogicalVariable> variables = ragg.getVariables();
         List<Mutable<ILogicalExpression>> expressions = ragg.getExpressions();
-        int[] outColumns = new int[variables.size()];
-        for (int i = 0; i < outColumns.length; i++) {
-            outColumns[i] = opSchema.findVariable(variables.get(i));
-        }
         IRunningAggregateEvaluatorFactory[] runningAggFuns = new IRunningAggregateEvaluatorFactory[expressions.size()];
         IExpressionRuntimeProvider expressionRuntimeProvider = context.getExpressionRuntimeProvider();
         for (int i = 0; i < runningAggFuns.length; i++) {
@@ -85,8 +81,30 @@
                     context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas, context);
         }
 
-        // TODO push projections into the operator
-        int[] projectionList = JobGenHelper.projectAllVariables(opSchema);
+        int[] outColumns = new int[variables.size()];
+        int[] projectionList;
+
+        if (ragg.isProjectPushed()) {
+            for (int i = 0; i < outColumns.length; i++) {
+                outColumns[i] = inputSchemas[0].getSize() + i;
+            }
+            List<LogicalVariable> projectVars = ragg.getProjectVariables();
+
+            projectionList = new int[projectVars.size()];
+            int c = 0;
+            for (LogicalVariable projectVar : projectVars) {
+                if (variables.contains(projectVar)) {
+                    projectionList[c++] = inputSchemas[0].getSize() + variables.indexOf(projectVar);
+                } else {
+                    projectionList[c++] = inputSchemas[0].findVariable(projectVar);
+                }
+            }
+        } else {
+            for (int i = 0; i < outColumns.length; i++) {
+                outColumns[i] = opSchema.findVariable(variables.get(i));
+            }
+            projectionList = JobGenHelper.projectAllVariables(opSchema);
+        }
 
         RunningAggregateRuntimeFactory runtime =
                 new RunningAggregateRuntimeFactory(projectionList, outColumns, runningAggFuns);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
index 5af7f98..5afe1d4 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
@@ -72,9 +72,11 @@
         boolean retainMissing = retainMissingAsValue != null;
         IMissingWriterFactory missingWriterFactory =
                 retainMissing ? JobGenHelper.getMissingWriterFactory(context, retainMissingAsValue) : null;
-        StreamSelectRuntimeFactory runtime =
-                new StreamSelectRuntimeFactory(cond, null, context.getBinaryBooleanInspectorFactory(), retainMissing,
-                        inputSchemas[0].findVariable(select.getMissingPlaceholderVariable()), missingWriterFactory);
+        StreamSelectRuntimeFactory runtime = new StreamSelectRuntimeFactory(cond,
+                select.isProjectPushed() ? JobGenHelper.projectVariables(inputSchemas[0], select.getProjectVariables())
+                        : null,
+                context.getBinaryBooleanInspectorFactory(), retainMissing,
+                inputSchemas[0].findVariable(select.getMissingPlaceholderVariable()), missingWriterFactory);
         runtime.setSourceLocation(select.getSourceLocation());
         // contribute one Asterix framewriter
         RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
index 8c50bcd..f904c56 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
@@ -187,6 +187,9 @@
     public Void visitRunningAggregateOperator(RunningAggregateOperator op, Integer indent) throws AlgebricksException {
         addIndent(indent).append("running-aggregate ").append(str(op.getVariables())).append(" <- ");
         pprintExprList(op.getExpressions(), indent);
+        if (op.isProjectPushed()) {
+            buffer.append(" project: ").append(str(op.getProjectVariables()));
+        }
         return null;
     }
 
@@ -261,6 +264,9 @@
     public Void visitAssignOperator(AssignOperator op, Integer indent) throws AlgebricksException {
         addIndent(indent).append("assign ").append(str(op.getVariables())).append(" <- ");
         pprintExprList(op.getExpressions(), indent);
+        if (op.isProjectPushed()) {
+            buffer.append(" project: ").append(str(op.getProjectVariables()));
+        }
         return null;
     }
 
@@ -298,7 +304,8 @@
                 ? " retain-untrue (" + op.getMissingPlaceholderVariable() + " <- " + op.getRetainMissingAsValue() + ")"
                 : "";
         addIndent(indent).append("select (").append(op.getCondition().getValue().accept(exprVisitor, indent))
-                .append(")").append(retainMissing);
+                .append(")").append(retainMissing)
+                .append(op.isProjectPushed() ? " project: " + str(op.getProjectVariables()) : "");
         return null;
     }
 
@@ -354,6 +361,9 @@
             buffer.append(" at " + op.getPositionalVariable());
         }
         buffer.append(" <- " + op.getExpressionRef().getValue().accept(exprVisitor, indent));
+        if (op.isProjectPushed()) {
+            buffer.append(" project: ").append(str(op.getProjectVariables()));
+        }
         return null;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
index 36c33e6..ee125bf 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
@@ -26,6 +26,7 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.Optional;
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -98,15 +99,34 @@
     private static final DefaultIndenter OBJECT_INDENT = new DefaultIndenter("   ", DefaultIndenter.SYS_LF);
     private static final String OPERATOR_FIELD = "operator";
     private static final String VARIABLES_FIELD = "variables";
+    private static final String PROJECT_VARIABLES_FIELD = "project-variables";
     // printing using the "expressions" field has to be an array of strings of the form ["str1", "str2", ...]
     private static final String EXPRESSIONS_FIELD = "expressions";
     private static final String EXPRESSION_FIELD = "expression";
     private static final String CONDITION_FIELD = "condition";
     private static final String MISSING_VALUE_FIELD = "missing-value";
     private static final String OPTIMIZER_ESTIMATES = "optimizer-estimates";
+
+    private static final String MIN_TIME = "min-time";
+
+    private static final String MAX_TIME = "max-time";
+
+    private static final String MIN_CARDINALITY = "min-cardinality";
+
+    private static final String MAX_CARDINALITY = "max-cardinality";
+
+    private static final String TOTAL_CARDINALITY = "total-cardinality";
+    private static final String NAME = "name";
+    private static final String ID = "id";
+    private static final String RUNTIME_ID = "runtime-id";
+
+    private static final String JOBLETS = "joblets";
+    private static final String CDID = "CDID";
+    private static final String ANID_START = "ANID:";
     private final Map<AbstractLogicalOperator, String> operatorIdentity = new HashMap<>();
     private Map<Object, String> log2odid = Collections.emptyMap();
     private Map<String, OperatorProfile> profile = Collections.emptyMap();
+
     private final IdCounter idCounter = new IdCounter();
     private final JsonGenerator jsonGenerator;
 
@@ -154,7 +174,7 @@
         }
     }
 
-    private class ExtendedActivityId {
+    private static class ExtendedActivityId {
         private final OperatorDescriptorId odId;
         private final int id;
         private final int microId;
@@ -162,7 +182,7 @@
         private final int subId;
 
         ExtendedActivityId(String str) {
-            if (str.startsWith("ANID:")) {
+            if (str.startsWith(ANID_START)) {
                 str = str.substring(5);
                 int idIdx = str.lastIndexOf(':');
                 this.odId = OperatorDescriptorId.parse(str.substring(0, idIdx));
@@ -201,11 +221,15 @@
 
         @Override
         public String toString() {
-            return "ANID:" + odId + ":" + getLocalId();
+            return "ANID:" + odId + ":" + id + "." + getLocalId();
+        }
+
+        private String getActivityAndLocalId() {
+            return "ANID:" + id + (!getLocalId().isEmpty() ? "." + getLocalId() : "");
         }
 
         private void catenateId(StringBuilder sb, int i) {
-            if (sb.length() == 0) {
+            if (sb.isEmpty()) {
                 sb.append(i);
                 return;
             }
@@ -213,9 +237,8 @@
             sb.append(i);
         }
 
-        public String getLocalId() {
+        private String getLocalId() {
             StringBuilder sb = new StringBuilder();
-            catenateId(sb, odId.getId());
             if (microId > 0) {
                 catenateId(sb, microId);
             }
@@ -227,63 +250,83 @@
         }
     }
 
+    private static class MinMax<T extends Comparable<T>> {
+        T min;
+        T max;
+
+        public MinMax(T initial) {
+            this.min = initial;
+            this.max = initial;
+        }
+
+        public void update(T val) {
+            min = min.compareTo(val) > 0 ? val : min;
+            max = max.compareTo(val) < 0 ? val : max;
+        }
+
+    }
+
     private class OperatorProfile {
-        Map<String, Pair<Double, Double>> activityTimes;
-        Map<String, Pair<Long, Long>> activityCards;
+
+        Map<String, String> activityNames;
+        Map<String, MinMax<Double>> activityTimes;
+        Map<String, MinMax<Long>> activityCards;
+        Map<String, Long> activityCardTotal;
 
         OperatorProfile() {
+            activityNames = new HashMap<>();
             activityTimes = new HashMap<>();
             activityCards = new HashMap<>();
+            activityCardTotal = new HashMap<>();
         }
 
-        void updateOperator(String extendedOpId, double time, long cardinality) {
-            updateMinMax(time, extendedOpId, activityTimes);
-            if (cardinality > 0) {
-                updateMinMax(cardinality, extendedOpId, activityCards);
-            }
+        void updateOperator(String extendedOpId, String name, double time, long cardinality) {
+            updateOperator(extendedOpId, name, time);
+            updateMinMax(cardinality, extendedOpId, activityCards);
+            activityCardTotal.compute(extendedOpId, (id, total) -> total == null ? cardinality : total + cardinality);
         }
 
-        void updateOperator(String extendedOpId, double time) {
+        void updateOperator(String extendedOpId, String name, double time) {
+            activityNames.put(extendedOpId, name);
             updateMinMax(time, extendedOpId, activityTimes);
         }
 
-        private <T extends Comparable<T>> void updateMinMax(T comp, String id, Map<String, Pair<T, T>> opMap) {
-            Pair<T, T> times = opMap.computeIfAbsent(id, i -> new Pair(comp, comp));
-            if (times.getFirst().compareTo(comp) > 0) {
-                times.setFirst(comp);
-            }
-            if (times.getSecond().compareTo(comp) < 0) {
-                times.setSecond(comp);
-            }
+        private <T extends Comparable<T>> void updateMinMax(T comp, String id, Map<String, MinMax<T>> opMap) {
+            MinMax<T> stat = opMap.computeIfAbsent(id, i -> new MinMax<>(comp));
+            stat.update(comp);
         }
     }
 
-    private ExtendedActivityId acIdFromName(String name) {
+    private Pair<ExtendedActivityId, String> splitAcId(String name) {
         String[] parts = name.split(" - ");
-        return new ExtendedActivityId(parts[0]);
+        return new Pair<>(new ExtendedActivityId(parts[0]), parts[1]);
     }
 
     Map<String, OperatorProfile> processProfile(ObjectNode profile) {
         Map<String, OperatorProfile> profiledOps = new HashMap<>();
-        for (JsonNode joblet : profile.get("joblets")) {
+        for (JsonNode joblet : profile.get(JOBLETS)) {
             for (JsonNode task : joblet.get("tasks")) {
                 for (JsonNode counters : task.get("counters")) {
-                    OperatorProfile info = profiledOps.computeIfAbsent(counters.get("runtime-id").asText(),
-                            i -> new OperatorProfile());
+                    if (counters.get(RUNTIME_ID).asText().contains(CDID)) {
+                        continue;
+                    }
+                    OperatorProfile info =
+                            profiledOps.computeIfAbsent(counters.get(RUNTIME_ID).asText(), i -> new OperatorProfile());
+                    Pair<ExtendedActivityId, String> identities = splitAcId(counters.get(NAME).asText());
                     JsonNode card = counters.get("cardinality-out");
                     if (card != null) {
-                        info.updateOperator(acIdFromName(counters.get("name").asText()).getLocalId(),
-                                counters.get("run-time").asDouble(), counters.get("cardinality-out").asLong(-1));
+                        info.updateOperator(identities.first.getActivityAndLocalId(), identities.second,
+                                counters.get("run-time").asDouble(), card.asLong(0));
+                    } else {
+                        info.updateOperator(identities.first.getActivityAndLocalId(), identities.second,
+                                counters.get("run-time").asDouble());
                     }
-                    info.updateOperator(acIdFromName(counters.get("name").asText()).getLocalId(),
-                            counters.get("run-time").asDouble());
                 }
                 for (JsonNode partition : task.get("partition-send-profile")) {
                     String id = partition.get("partition-id").get("connector-id").asText();
                     OperatorProfile info = profiledOps.computeIfAbsent(id, i -> new OperatorProfile());
-                    //CDIDs are unique
-                    info.updateOperator("0",
-                            partition.get("close-time").asDouble() - partition.get("open-time").asDouble());
+                    //TODO: connector times need to be calculated accurately, until then they won't be included
+                    info.updateOperator("0", "", 0, partition.get("cardinality").asLong(0));
                 }
             }
         }
@@ -350,6 +393,45 @@
         }
     }
 
+    private void printActivityStats(Optional<MinMax<Double>> time, Optional<MinMax<Long>> card,
+            Optional<Long> totalCard) throws IOException {
+        if (time.isPresent()) {
+            jsonGenerator.writeNumberField(MIN_TIME, time.get().min);
+            jsonGenerator.writeNumberField(MAX_TIME, time.get().max);
+        }
+        if (card.isPresent()) {
+            jsonGenerator.writeNumberField(MIN_CARDINALITY, card.get().min);
+            jsonGenerator.writeNumberField(MAX_CARDINALITY, card.get().max);
+        }
+        if (totalCard.isPresent()) {
+            jsonGenerator.writeNumberField(TOTAL_CARDINALITY, totalCard.get());
+        }
+    }
+
+    private void printOperatorStats(OperatorProfile info) throws IOException {
+        if (info.activityTimes.size() == 1) {
+            Optional<MinMax<Double>> times = info.activityTimes.values().stream().findFirst();
+            Optional<MinMax<Long>> cards = info.activityCards.values().stream().findFirst();
+            Optional<Long> total = info.activityCardTotal.values().stream().findFirst();
+            printActivityStats(times, cards, total);
+        } else if (info.activityTimes.size() > 1) {
+            jsonGenerator.writeArrayFieldStart("activity-stats");
+            for (String acId : info.activityTimes.keySet()) {
+                jsonGenerator.writeStartObject();
+                String prettyName = info.activityNames.get(acId);
+                if (prettyName != null) {
+                    jsonGenerator.writeStringField(NAME, prettyName);
+                }
+                jsonGenerator.writeStringField("id", acId);
+                printActivityStats(Optional.ofNullable(info.activityTimes.get(acId)),
+                        Optional.ofNullable(info.activityCards.get(acId)),
+                        Optional.ofNullable(info.activityCardTotal.get(acId)));
+                jsonGenerator.writeEndObject();
+            }
+            jsonGenerator.writeEndArray();
+        }
+    }
+
     private void printOperatorImpl(AbstractLogicalOperator op, boolean printInputs, boolean printOptimizerEstimates)
             throws AlgebricksException {
         try {
@@ -358,34 +440,10 @@
             jsonGenerator.writeStringField("operatorId", idCounter.printOperatorId(op));
             String od = log2odid.get(op);
             if (od != null) {
-                jsonGenerator.writeStringField("runtime-id", od);
+                jsonGenerator.writeStringField(RUNTIME_ID, od);
                 OperatorProfile info = profile.get(od);
                 if (info != null) {
-                    if (info.activityTimes.size() == 1) {
-                        Pair<Double, Double> minMax = info.activityTimes.values().iterator().next();
-                        jsonGenerator.writeNumberField("min-time", minMax.first);
-                        jsonGenerator.writeNumberField("max-time", minMax.second);
-                        if (info.activityCards.size() > 0) {
-                            Pair<Long, Long> minMaxCard = info.activityCards.values().iterator().next();
-                            jsonGenerator.writeNumberField("min-cardinality", minMaxCard.first);
-                            jsonGenerator.writeNumberField("max-cardinality", minMaxCard.second);
-                        }
-                    } else {
-                        jsonGenerator.writeObjectFieldStart("times");
-                        for (String acId : info.activityTimes.keySet()) {
-                            jsonGenerator.writeObjectFieldStart(acId);
-                            jsonGenerator.writeNumberField("min-time", info.activityTimes.get(acId).first);
-                            jsonGenerator.writeNumberField("max-time", info.activityTimes.get(acId).second);
-                            Pair<Long, Long> cards = info.activityCards.get(acId);
-                            if (cards != null) {
-                                jsonGenerator.writeNumberField("min-cardinality", info.activityCards.get(acId).first);
-                                jsonGenerator.writeNumberField("max-cardinality", info.activityCards.get(acId).second);
-                            }
-                            jsonGenerator.writeEndObject();
-                        }
-                        jsonGenerator.writeEndObject();
-                    }
-
+                    printOperatorStats(info);
                 }
             }
             IPhysicalOperator pOp = op.getPhysicalOperator();
@@ -467,6 +525,9 @@
         try {
             jsonGenerator.writeStringField(OPERATOR_FIELD, "running-aggregate");
             writeVariablesAndExpressions(op.getVariables(), op.getExpressions(), indent);
+            if (op.isProjectPushed()) {
+                writeArrayFieldOfVariables(PROJECT_VARIABLES_FIELD, op.getProjectVariables());
+            }
             return null;
         } catch (IOException e) {
             throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
@@ -576,6 +637,9 @@
         try {
             jsonGenerator.writeStringField(OPERATOR_FIELD, "assign");
             writeVariablesAndExpressions(op.getVariables(), op.getExpressions(), indent);
+            if (op.isProjectPushed()) {
+                writeArrayFieldOfVariables(PROJECT_VARIABLES_FIELD, op.getProjectVariables());
+            }
             return null;
         } catch (IOException e) {
             throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
@@ -626,6 +690,9 @@
         try {
             jsonGenerator.writeStringField(OPERATOR_FIELD, "select");
             writeStringFieldExpression(CONDITION_FIELD, op.getCondition(), indent);
+            if (op.isProjectPushed()) {
+                writeArrayFieldOfVariables(PROJECT_VARIABLES_FIELD, op.getProjectVariables());
+            }
             return null;
         } catch (IOException e) {
             throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
@@ -1033,6 +1100,9 @@
                 jsonGenerator.writeStringField("position", String.valueOf(positionalVariable));
             }
             writeArrayFieldOfExpression(EXPRESSIONS_FIELD, op.getExpressionRef(), indent);
+            if (op.isProjectPushed()) {
+                writeArrayFieldOfVariables(PROJECT_VARIABLES_FIELD, op.getProjectVariables());
+            }
         } catch (IOException e) {
             throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
         }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
index 98c4223..ccb568c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
@@ -46,4 +46,6 @@
             StorageUtil.getIntSizeInBytes(8, StorageUtil.StorageUnit.KILOBYTE);
     public static final boolean BATCH_LOOKUP_DEFAULT = true;
     public static final boolean COLUMN_FILTER_DEFAULT = true;
+    public static final boolean ORDERED_FIELDS = true;
+    public static final int MAX_VARIABLE_OCCURRENCES_INLINING_DEFAULT = 128;
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
index 11171a1..7419964 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
@@ -62,6 +62,9 @@
     private static final String MIN_JOIN_FRAMES = "MIN_JOIN_FRAMES";
     private static final String MIN_GROUP_FRAMES = "MIN_GROUP_FRAMES";
     private static final String MIN_WINDOW_FRAMES = "MIN_WINDOW_FRAMES";
+    private static final String MAX_VARIABLE_OCCURRENCES_INLINING = "MAX_VARIABLE_OCCURRENCES_INLINING";
+
+    private static final String ORDER_FIELDS = "ORDERED_FIELDS";
 
     private final Properties properties = new Properties();
 
@@ -159,6 +162,10 @@
         setInt(MAX_FRAMES_FOR_TEXTSEARCH, frameLimit);
     }
 
+    public void setOrderFields(boolean orderFields) {
+        setBoolean(ORDER_FIELDS, orderFields);
+    }
+
     public int getHashGroupByTableSize() {
         return getInt(DEFAULT_HASH_GROUP_TABLE_SIZE, 10485767);
     }
@@ -235,6 +242,10 @@
         return getBoolean(SORT_PARALLEL, AlgebricksConfig.SORT_PARALLEL_DEFAULT);
     }
 
+    public boolean isOrderField() {
+        return getBoolean(ORDER_FIELDS, AlgebricksConfig.ORDERED_FIELDS);
+    }
+
     public void setSortParallel(boolean sortParallel) {
         setBoolean(SORT_PARALLEL, sortParallel);
     }
@@ -383,6 +394,14 @@
         return properties.getProperty(property);
     }
 
+    public int getMaxVariableOccurrencesForInlining() {
+        return getInt(MAX_VARIABLE_OCCURRENCES_INLINING, AlgebricksConfig.MAX_VARIABLE_OCCURRENCES_INLINING_DEFAULT);
+    }
+
+    public void setMaxVariableOccurrencesForInlining(int maxVariableOccurrencesForInlining) {
+        setInt(MAX_VARIABLE_OCCURRENCES_INLINING, maxVariableOccurrencesForInlining);
+    }
+
     private void setInt(String property, int value) {
         properties.setProperty(property, Integer.toString(value));
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
index 1f36aa5..4664a7e 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
@@ -112,6 +112,9 @@
         stringBuilder.setLength(0);
         stringBuilder.append("running-aggregate ").append(str(op.getVariables())).append(" <- ");
         printExprList(op.getExpressions());
+        if (op.isProjectPushed()) {
+            stringBuilder.append(" project: ").append(str(op.getProjectVariables()));
+        }
         appendSchema(op, showDetails);
         appendAnnotations(op, showDetails);
         appendPhysicalOperatorInfo(op, showDetails);
@@ -221,6 +224,10 @@
         stringBuilder.setLength(0);
         stringBuilder.append("assign ").append(str(op.getVariables())).append(" <- ");
         printExprList(op.getExpressions());
+        if (op.isProjectPushed()) {
+            stringBuilder.append(" project: ").append(str(op.getProjectVariables()));
+        }
+
         appendSchema(op, showDetails);
         appendAnnotations(op, showDetails);
         appendPhysicalOperatorInfo(op, showDetails);
@@ -268,6 +275,9 @@
     public String visitSelectOperator(SelectOperator op, Boolean showDetails) {
         stringBuilder.setLength(0);
         stringBuilder.append("select (").append(op.getCondition().getValue().toString()).append(")");
+        if (op.isProjectPushed()) {
+            stringBuilder.append(" project: ").append(str(op.getProjectVariables()));
+        }
         appendSchema(op, showDetails);
         appendAnnotations(op, showDetails);
         appendPhysicalOperatorInfo(op, showDetails);
@@ -343,6 +353,9 @@
             stringBuilder.append(" at ").append(op.getPositionalVariable());
         }
         stringBuilder.append(" <- ").append(op.getExpressionRef().getValue().toString());
+        if (op.isProjectPushed()) {
+            stringBuilder.append(" project: ").append(str(op.getProjectVariables()));
+        }
         appendSchema(op, showDetails);
         appendAnnotations(op, showDetails);
         appendPhysicalOperatorInfo(op, showDetails);
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EmbedProjectRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EmbedProjectRule.java
new file mode 100644
index 0000000..4acb509
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EmbedProjectRule.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.algebricks.rewriter.rules;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractProjectingOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+
+ * This rule should run last during optimization. The assumption is that the rules set that includes this rule will run
+ * only once (i.e. sequential once).
+ */
+
+public class EmbedProjectRule implements IAlgebraicRewriteRule {
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        if (op.getOperatorTag() != LogicalOperatorTag.PROJECT) {
+            return false;
+        }
+
+        Mutable<ILogicalOperator> opRef2 = op.getInputs().get(0);
+        AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getValue();
+
+        if (!isProjectable(op2.getOperatorTag())) {
+            return false;
+        }
+
+        ProjectOperator projectOperator = (ProjectOperator) op;
+
+        AbstractProjectingOperator projectPushableOperator = (AbstractProjectingOperator) op2;
+        projectPushableOperator.pushProjectionVariables(projectOperator.getVariables());
+
+        opRef.setValue(op2);
+        return true;
+    }
+
+    private static boolean isProjectable(LogicalOperatorTag op) {
+        return switch (op) {
+            case ASSIGN, UNNEST, LEFT_OUTER_UNNEST, RUNNINGAGGREGATE, SELECT -> true;
+            default -> false;
+        };
+    }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
index 6ad50e6..5970f4f 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.algebricks.rewriter.rules;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -84,6 +85,7 @@
     private final List<LogicalVariable> usedVars = new ArrayList<>();
     // map of variables and the counts of how many times they were used
     private final Map<LogicalVariable, MutableInt> usedVariableCounter = new HashMap<>();
+    private final Map<LogicalVariable, Map<LogicalVariable, Integer>> totalLeafVariableCounter = new HashMap<>();
 
     @Override
     public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
@@ -113,6 +115,7 @@
         inlineVisitor.setContext(context);
         subTreesDone.clear();
         usedVariableCounter.clear();
+        totalLeafVariableCounter.clear();
     }
 
     protected boolean performBottomUpAction(ILogicalOperator op) throws AlgebricksException {
@@ -168,6 +171,20 @@
             }
         }
 
+        if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+            AssignOperator assignOp = (AssignOperator) op;
+            computeLeafVariablesCount(assignOp);
+            List<LogicalVariable> vars = assignOp.getVariables();
+            for (LogicalVariable variable : vars) {
+                // Don't inline variables that potentially reference a large number of the same leaf variable.
+                Map<LogicalVariable, Integer> varMap = totalLeafVariableCounter.get(variable);
+                if (varMap != null && !varMap.isEmpty() && Collections.max(varMap.values()) > context
+                        .getPhysicalOptimizationConfig().getMaxVariableOccurrencesForInlining()) {
+                    varAssignRhs.remove(variable);
+                }
+            }
+        }
+
         // Descend into subplan
         if (op.getOperatorTag() == LogicalOperatorTag.SUBPLAN || op.getOperatorTag() == LogicalOperatorTag.WINDOW) {
             List<ILogicalPlan> nestedPlans = ((AbstractOperatorWithNestedPlans) op).getNestedPlans();
@@ -259,6 +276,28 @@
         }
     }
 
+    private void computeLeafVariablesCount(AssignOperator assignOp) {
+        List<LogicalVariable> vars = assignOp.getVariables();
+        List<Mutable<ILogicalExpression>> exprs = assignOp.getExpressions();
+        for (int i = 0; i < vars.size(); i++) {
+            LogicalVariable variable = vars.get(i);
+            ILogicalExpression expr = exprs.get(i).getValue();
+            usedVars.clear();
+            expr.getUsedVariables(usedVars);
+            Map<LogicalVariable, Integer> varMap =
+                    totalLeafVariableCounter.computeIfAbsent(variable, k -> new HashMap<>());
+            for (LogicalVariable usedVar : usedVars) {
+                if (totalLeafVariableCounter.containsKey(usedVar)) {
+                    for (Map.Entry<LogicalVariable, Integer> entry : totalLeafVariableCounter.get(usedVar).entrySet()) {
+                        varMap.put(entry.getKey(), entry.getValue() + varMap.getOrDefault(entry.getKey(), 0));
+                    }
+                } else {
+                    varMap.put(usedVar, 1);
+                }
+            }
+        }
+    }
+
     public static class InlineVariablesVisitor extends LogicalExpressionReferenceTransformVisitor
             implements ILogicalExpressionReferenceTransform {
 
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushProjectIntoDataSourceScanRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushProjectIntoDataSourceScanRule.java
index fedc420..46d187c 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushProjectIntoDataSourceScanRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushProjectIntoDataSourceScanRule.java
@@ -53,7 +53,7 @@
             return false;
         DataSourceScanOperator scanOp = (DataSourceScanOperator) inputOp;
         ProjectOperator projectOp = (ProjectOperator) project;
-        scanOp.addProjectVariables(projectOp.getVariables());
+        scanOp.pushProjectionVariables(projectOp.getVariables());
         if (op.getOperatorTag() != LogicalOperatorTag.EXCHANGE) {
             op.getInputs().set(0, project.getInputs().get(0));
         } else {
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/ProfiledPushRuntime.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/ProfiledPushRuntime.java
index 354f172..bf4533f 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/ProfiledPushRuntime.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/ProfiledPushRuntime.java
@@ -70,7 +70,7 @@
     public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
         if (writer instanceof ITimedWriter) {
             ITimedWriter wrapper = (ITimedWriter) writer;
-            wrapper.setUpstreamStats(stats);
+            wrapper.setInputStats(stats);
             outputs.put(index, wrapper);
         }
         wrapped.setOutputFrameWriter(index, writer, recordDesc);
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFieldFramePushRuntime.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFieldFramePushRuntime.java
index 9b32aa1..1592e02 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFieldFramePushRuntime.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFieldFramePushRuntime.java
@@ -43,7 +43,7 @@
 
     protected final void initAccessAppendFieldRef(IHyracksTaskContext ctx) throws HyracksDataException {
         frame = new VSizeFrame(ctx);
-        appender = new FrameFixedFieldTupleAppender(inputRecordDesc.getFieldCount());
+        appender = new FrameFixedFieldTupleAppender(outputRecordDesc.getFieldCount());
         appender.reset(frame, true);
         tAccess = new FrameTupleAccessor(inputRecordDesc);
         tRef = new FrameTupleReference();
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/AlgebricksMetaOperatorDescriptor.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/AlgebricksMetaOperatorDescriptor.java
index 2a3fa7e..f6d9cd1 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/AlgebricksMetaOperatorDescriptor.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/AlgebricksMetaOperatorDescriptor.java
@@ -265,7 +265,7 @@
             }
 
             @Override
-            public void setUpstreamStats(IOperatorStats stats) {
+            public void setInputStats(IOperatorStats stats) {
                 parentStats = stats;
             }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java
index a39cd05..6cd7a05 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java
@@ -92,7 +92,9 @@
             }
             sb.append(evalFactories[i].toString());
         }
-        sb.append("]");
+        sb.append("] ");
+        sb.append(" project: ");
+        sb.append(Arrays.toString(projectionList));
         return sb.toString();
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamSelectRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamSelectRuntimeFactory.java
index 5cfebc0..08fa2c9 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamSelectRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamSelectRuntimeFactory.java
@@ -20,6 +20,7 @@
 
 import java.io.DataOutput;
 import java.nio.ByteBuffer;
+import java.util.Arrays;
 
 import org.apache.hyracks.algebricks.data.IBinaryBooleanInspector;
 import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
@@ -63,7 +64,7 @@
 
     @Override
     public String toString() {
-        return "stream-select " + cond.toString();
+        return "stream-select " + cond.toString() + " project: " + Arrays.toString(projectionList);
     }
 
     @Override
@@ -158,11 +159,23 @@
         }
 
         protected void retainMissingTuple(int t) throws HyracksDataException {
-            for (int i = 0; i < tRef.getFieldCount(); i++) {
-                if (i == missingPlaceholderVariableIndex) {
+            if (projectionList == null) {
+                for (int i = 0; i < tRef.getFieldCount(); i++) {
+                    if (i == missingPlaceholderVariableIndex) {
+                        appendField(missingTupleBuilder.getByteArray(), 0, missingTupleBuilder.getSize());
+                    } else {
+                        appendField(tAccess, t, i);
+                    }
+                }
+                return;
+            }
+
+            for (int i = 0; i < projectionList.length; i++) {
+                int index = projectionList[i];
+                if (index == missingPlaceholderVariableIndex) {
                     appendField(missingTupleBuilder.getByteArray(), 0, missingTupleBuilder.getSize());
                 } else {
-                    appendField(tAccess, t, i);
+                    appendField(tAccess, t, index);
                 }
             }
         }
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java
index 1b42bb6..c92cab1 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java
@@ -21,6 +21,7 @@
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.Arrays;
 
 import org.apache.hyracks.algebricks.data.IUnnestingPositionWriter;
 import org.apache.hyracks.algebricks.data.IUnnestingPositionWriterFactory;
@@ -41,34 +42,35 @@
 
 public class UnnestRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
 
+    /**
+     * @param projection[i] is the input field index of the i-th field in the output tuple
+     * @param projection[i] should be @param outCol if i-th field is the unnest field
+     * @param projection[i] should be @param positionalCol if i-th field is the positional field
+     * @param outCol,positionalCol can be -1 if the output tuple does not contain the unnest/positional field
+     * @param outCol,positionalCol should not be in the input field index range to avoid ambiguity
+     */
+
     private static final long serialVersionUID = 1L;
 
     private final int outCol;
+    private final int positionalCol;
     private final IUnnestingEvaluatorFactory unnestingFactory;
-    private final boolean unnestColIsProjected;
     private final IUnnestingPositionWriterFactory positionWriterFactory;
     private final boolean leftOuter;
     private final IMissingWriterFactory missingWriterFactory;
-    private int outColPos;
 
     public UnnestRuntimeFactory(int outCol, IUnnestingEvaluatorFactory unnestingFactory, int[] projectionList,
             boolean leftOuter, IMissingWriterFactory missingWriterFactory) {
-        this(outCol, unnestingFactory, projectionList, null, leftOuter, missingWriterFactory);
+        this(outCol, -1, unnestingFactory, projectionList, null, leftOuter, missingWriterFactory);
     }
 
-    public UnnestRuntimeFactory(int outCol, IUnnestingEvaluatorFactory unnestingFactory, int[] projectionList,
-            IUnnestingPositionWriterFactory positionWriterFactory, boolean leftOuter,
+    public UnnestRuntimeFactory(int outCol, int positionalCol, IUnnestingEvaluatorFactory unnestingFactory,
+            int[] projectionList, IUnnestingPositionWriterFactory positionWriterFactory, boolean leftOuter,
             IMissingWriterFactory missingWriterFactory) {
         super(projectionList);
         this.outCol = outCol;
+        this.positionalCol = positionalCol;
         this.unnestingFactory = unnestingFactory;
-        outColPos = -1;
-        for (int f = 0; f < projectionList.length; f++) {
-            if (projectionList[f] == outCol) {
-                outColPos = f;
-            }
-        }
-        unnestColIsProjected = outColPos >= 0;
         this.positionWriterFactory = positionWriterFactory;
         this.leftOuter = leftOuter;
         this.missingWriterFactory = missingWriterFactory;
@@ -76,7 +78,8 @@
 
     @Override
     public String toString() {
-        return "unnest " + outCol + " <- " + unnestingFactory;
+        return "unnest " + outCol + (positionalCol >= 0 ? " at " + positionalCol : "") + " <- " + unnestingFactory
+                + " project: " + Arrays.toString(projectionList);
     }
 
     @Override
@@ -134,34 +137,24 @@
 
             private void writeOutput(int t, int positionIndex, boolean missing)
                     throws HyracksDataException, IOException {
-                if (!unnestColIsProjected && positionWriter == null) {
-                    appendProjectionToFrame(t, projectionList);
-                    appendToFrameFromTupleBuilder(tupleBuilder);
-                    return;
-                }
-
                 tupleBuilder.reset();
-                for (int f = 0; f < outColPos; f++) {
-                    tupleBuilder.addField(tAccess, t, f);
-                }
-                if (unnestColIsProjected) {
-                    if (missing) {
-                        tupleBuilder.addField(missingBytes.getByteArray(), 0, missingBytes.size());
+                for (int f = 0; f < projectionList.length; f++) {
+                    int col = projectionList[f];
+                    if (col == outCol) {
+                        if (missing) {
+                            tupleBuilder.addField(missingBytes.getByteArray(), 0, missingBytes.size());
+                        } else {
+                            tupleBuilder.addField(p.getByteArray(), p.getStartOffset(), p.getLength());
+                        }
+                    } else if (col == positionalCol) {
+                        if (missing) {
+                            tupleBuilder.addField(missingBytes.getByteArray(), 0, missingBytes.size());
+                        } else {
+                            positionWriter.write(tupleBuilder.getDataOutput(), positionIndex);
+                            tupleBuilder.addFieldEndOffset();
+                        }
                     } else {
-                        tupleBuilder.addField(p.getByteArray(), p.getStartOffset(), p.getLength());
-                    }
-                }
-                for (int f = unnestColIsProjected ? outColPos + 1 : outColPos; f < (positionWriter != null
-                        ? projectionList.length - 1 : projectionList.length); f++) {
-                    tupleBuilder.addField(tAccess, t, f);
-                }
-                if (positionWriter != null) {
-                    // Write the positional variable
-                    if (missing) {
-                        tupleBuilder.addField(missingBytes.getByteArray(), 0, missingBytes.size());
-                    } else {
-                        positionWriter.write(tupleBuilder.getDataOutput(), positionIndex);
-                        tupleBuilder.addFieldEndOffset();
+                        tupleBuilder.addField(tAccess, t, projectionList[f]);
                     }
                 }
                 appendToFrameFromTupleBuilder(tupleBuilder);
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/IWriterPartitioner.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/IWriterPartitioner.java
index cb72aec..7cf4d05 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/IWriterPartitioner.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/IWriterPartitioner.java
@@ -21,7 +21,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 
-interface IWriterPartitioner {
+public interface IWriterPartitioner {
     boolean isNewPartition(FrameTupleAccessor tupleAccessor, int index) throws HyracksDataException;
 
 }
\ No newline at end of file
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/SinkExternalWriterRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/SinkExternalWriterRuntimeFactory.java
index 321828f..3612511 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/SinkExternalWriterRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/SinkExternalWriterRuntimeFactory.java
@@ -23,7 +23,6 @@
 import org.apache.hyracks.algebricks.runtime.writers.IExternalWriter;
 import org.apache.hyracks.algebricks.runtime.writers.IExternalWriterFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -31,64 +30,35 @@
 public final class SinkExternalWriterRuntimeFactory extends AbstractPushRuntimeFactory {
     private static final long serialVersionUID = -2215789207336628581L;
     private final int sourceColumn;
-    private final int[] partitionColumns;
-    private final IBinaryComparatorFactory[] partitionComparatorFactories;
-    private final boolean partitionByKey;
     private final RecordDescriptor inputRecordDescriptor;
     private final IExternalWriterFactory writerFactory;
+    private final WriterPartitionerFactory partitionerFactory;
 
     public SinkExternalWriterRuntimeFactory(int sourceColumn, int[] partitionColumns,
             IBinaryComparatorFactory[] partitionComparatorFactories, RecordDescriptor inputRecordDescriptor,
             IExternalWriterFactory writerFactory) {
-        this(sourceColumn, partitionColumns, partitionComparatorFactories, false, inputRecordDescriptor, writerFactory);
+        this(sourceColumn, inputRecordDescriptor, writerFactory,
+                new WriterPartitionerFactory(partitionColumns, partitionComparatorFactories));
     }
 
     public SinkExternalWriterRuntimeFactory(int sourceColumn, RecordDescriptor inputRecordDescriptor,
             IExternalWriterFactory writerFactory) {
-        this(sourceColumn, null, null, true, inputRecordDescriptor, writerFactory);
+        this(sourceColumn, inputRecordDescriptor, writerFactory, new WriterPartitionerFactory());
     }
 
-    private SinkExternalWriterRuntimeFactory(int sourceColumn, int[] partitionColumns,
-            IBinaryComparatorFactory[] partitionComparatorFactories, boolean partitionByKey,
-            RecordDescriptor inputRecordDescriptor, IExternalWriterFactory writerFactory) {
+    private SinkExternalWriterRuntimeFactory(int sourceColumn, RecordDescriptor inputRecordDescriptor,
+            IExternalWriterFactory writerFactory, WriterPartitionerFactory partitionerFactory) {
         this.sourceColumn = sourceColumn;
-        this.partitionColumns = partitionColumns;
-        this.partitionComparatorFactories = partitionComparatorFactories;
-        this.partitionByKey = partitionByKey;
         this.inputRecordDescriptor = inputRecordDescriptor;
         this.writerFactory = writerFactory;
+        this.partitionerFactory = partitionerFactory;
     }
 
     @Override
     public IPushRuntime[] createPushRuntime(IHyracksTaskContext ctx) throws HyracksDataException {
         IExternalWriter writer = writerFactory.createWriter(ctx);
-        SinkExternalWriterRuntime runtime =
-                new SinkExternalWriterRuntime(sourceColumn, createPartitioner(), inputRecordDescriptor, writer);
+        SinkExternalWriterRuntime runtime = new SinkExternalWriterRuntime(sourceColumn,
+                partitionerFactory.createPartitioner(), inputRecordDescriptor, writer);
         return new IPushRuntime[] { runtime };
     }
-
-    /**
-     * Creates the writer partitioner based on the provided parameters
-     *
-     * @return writer partitioner
-     */
-    private IWriterPartitioner createPartitioner() {
-        // key writer partitioner
-        if (partitionByKey) {
-            return KeyWriterPartitioner.INSTANCE;
-        }
-
-        // writer partitioner
-        if (partitionColumns.length > 0) {
-            IBinaryComparator[] comparators = new IBinaryComparator[partitionComparatorFactories.length];
-            for (int i = 0; i < partitionComparatorFactories.length; i++) {
-                comparators[i] = partitionComparatorFactories[i].createBinaryComparator();
-            }
-
-            return new WriterPartitioner(partitionColumns, comparators);
-        }
-
-        // no-op partitioner
-        return new NoOpWriterPartitioner();
-    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/WriterPartitionerFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/WriterPartitionerFactory.java
new file mode 100644
index 0000000..1d5b089
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/WriterPartitionerFactory.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.algebricks.runtime.operators.writer;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class WriterPartitionerFactory implements Serializable {
+    private static final long serialVersionUID = 8971234908711239L;
+    private final boolean partitionByKey;
+    private final int[] partitionColumns;
+    private final IBinaryComparatorFactory[] partitionComparatorFactories;
+
+    public WriterPartitionerFactory(int[] partitionColumns, IBinaryComparatorFactory[] partitionComparatorFactories) {
+        this(partitionColumns, partitionComparatorFactories, false);
+    }
+
+    public WriterPartitionerFactory() {
+        this(null, null, true);
+    }
+
+    private WriterPartitionerFactory(int[] partitionColumns, IBinaryComparatorFactory[] partitionComparatorFactories,
+            boolean partitionByKey) {
+        this.partitionColumns = partitionColumns;
+        this.partitionComparatorFactories = partitionComparatorFactories;
+        this.partitionByKey = partitionByKey;
+    }
+
+    /**
+     * Creates the writer partitioner based on the provided parameters
+     *
+     * @return writer partitioner
+     */
+    public IWriterPartitioner createPartitioner() {
+        // key writer partitioner
+        if (partitionByKey) {
+            return KeyWriterPartitioner.INSTANCE;
+        }
+
+        // writer partitioner
+        if (partitionColumns.length > 0) {
+            IBinaryComparator[] comparators = new IBinaryComparator[partitionComparatorFactories.length];
+            for (int i = 0; i < partitionComparatorFactories.length; i++) {
+                comparators[i] = partitionComparatorFactories[i].createBinaryComparator();
+            }
+
+            return new WriterPartitioner(partitionColumns, comparators);
+        }
+
+        // no-op partitioner
+        return new NoOpWriterPartitioner();
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ITimedWriter.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ITimedWriter.java
index 7b0f8c8..e4f990f 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ITimedWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ITimedWriter.java
@@ -22,7 +22,7 @@
 import org.apache.hyracks.api.job.profiling.IOperatorStats;
 
 public interface ITimedWriter extends IFrameWriter {
-    void setUpstreamStats(IOperatorStats stats);
+    void setInputStats(IOperatorStats stats);
 
     long getTotalTime();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ProfiledFrameWriter.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ProfiledFrameWriter.java
index cfb0e7b..fb4ed3c 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ProfiledFrameWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ProfiledFrameWriter.java
@@ -41,11 +41,12 @@
 
     // The downstream data consumer of this writer.
     private final IFrameWriter writer;
-    protected IOperatorStats upstreamStats = NoOpOperatorStats.INSTANCE;
+    protected IOperatorStats inputStats = NoOpOperatorStats.INSTANCE;
     private int minSz = Integer.MAX_VALUE;
     private int maxSz = -1;
     private long avgSz;
-    private ICounter totalTime;
+
+    private final ICounter totalTime;
 
     public ProfiledFrameWriter(IFrameWriter writer) {
         this.writer = writer;
@@ -53,8 +54,8 @@
     }
 
     @Override
-    public void setUpstreamStats(IOperatorStats stats) {
-        this.upstreamStats = stats;
+    public void setInputStats(IOperatorStats stats) {
+        this.inputStats = stats;
     }
 
     public static void timeMethod(HyracksRunnable r, ICounter c) throws HyracksDataException {
@@ -78,14 +79,14 @@
     }
 
     @Override
-    public final void open() throws HyracksDataException {
+    public void open() throws HyracksDataException {
         timeMethod(writer::open, totalTime);
     }
 
     private void updateTupleStats(ByteBuffer buffer) {
         int tupleCountOffset = FrameHelper.getTupleCountOffset(buffer.limit());
         int tupleCount = IntSerDeUtils.getInt(buffer.array(), tupleCountOffset);
-        ICounter tupleCounter = upstreamStats.getTupleCounter();
+        ICounter tupleCounter = inputStats.getTupleCounter();
         long prevCount = tupleCounter.get();
         for (int i = 0; i < tupleCount; i++) {
             int tupleLen = getTupleLength(i, tupleCountOffset, buffer);
@@ -99,25 +100,25 @@
             avgSz = (prev + tupleLen) / (prevCount + 1);
             prevCount++;
         }
-        upstreamStats.getMaxTupleSz().set(maxSz);
-        upstreamStats.getMinTupleSz().set(minSz);
-        upstreamStats.getAverageTupleSz().set(avgSz);
+        inputStats.getMaxTupleSz().set(maxSz);
+        inputStats.getMinTupleSz().set(minSz);
+        inputStats.getAverageTupleSz().set(avgSz);
         tupleCounter.update(tupleCount);
     }
 
     @Override
-    public final void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
         updateTupleStats(buffer);
         timeMethod(writer::nextFrame, buffer);
     }
 
     @Override
-    public final void flush() throws HyracksDataException {
+    public void flush() throws HyracksDataException {
         timeMethod(writer::flush, totalTime);
     }
 
     @Override
-    public final void fail() throws HyracksDataException {
+    public void fail() throws HyracksDataException {
         timeMethod(writer::fail, totalTime);
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ProfiledOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ProfiledOperatorNodePushable.java
index 1984d8e..cb188c0 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ProfiledOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ProfiledOperatorNodePushable.java
@@ -77,9 +77,9 @@
         if (writer instanceof ITimedWriter) {
             ITimedWriter wrapper = (ITimedWriter) writer;
             if (op instanceof ISelfProfilingNodePushable) {
-                wrapper.setUpstreamStats(((ISelfProfilingNodePushable) op).getStats());
+                wrapper.setInputStats(((ISelfProfilingNodePushable) op).getStats());
             } else {
-                wrapper.setUpstreamStats(stats);
+                wrapper.setInputStats(stats);
             }
             outputs.put(index, wrapper);
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
index 59a4da4..7cb107d 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
@@ -158,6 +158,10 @@
     UNSUPPORTED_WRITE_SPEC(128),
     JOB_REJECTED(129),
     FRAME_BIGGER_THAN_SORT_MEMORY(130),
+    RESULT_DOES_NOT_FOLLOW_SCHEMA(131),
+    EXTRA_FIELD_IN_RESULT_NOT_FOUND_IN_SCHEMA(132),
+    EMPTY_TYPE_INFERRED(133),
+    SCHEMA_LIMIT_EXCEEDED(134),
 
     // Compilation error codes.
     RULECOLLECTION_NOT_INSTANCE_OF_LIST(10000),
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java
index a6520c6..75cd23d 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java
@@ -24,6 +24,7 @@
 import java.nio.channels.WritableByteChannel;
 import java.util.List;
 import java.util.Set;
+import java.util.function.Predicate;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -296,4 +297,12 @@
      * @param bulkOperation the operation to perform
      */
     void performBulkOperation(IIOBulkOperation bulkOperation) throws HyracksDataException;
+
+    /**
+     * Lists the storage files and apply the provided predicate to calculate the total size of the resource
+     *
+     * @param relativePathFilter predicate
+     * @return total size of resource accepting the predicate
+     */
+    long getSize(Predicate<String> relativePathFilter);
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java
index 5045c86..4022961 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java
@@ -332,6 +332,10 @@
 
     public static String toString(byte[] bytes, int start) {
         StringBuilder builder = new StringBuilder();
+        return toString(bytes, start, builder);
+    }
+
+    public static String toString(byte[] bytes, int start, StringBuilder builder) {
         return toString(builder, bytes, start).toString();
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
index e94c12e..e1fbe30 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
@@ -148,6 +148,10 @@
 128 = Unsupported copy to specification: PARTITION BY %1$s, ORDER BY %2$s
 129 = Job %1$s failed to run. Cluster is not accepting jobs.
 130 = Frame data=%1$s (requiring %2$s) is bigger than the sort budget. Used=%3$s, max=%4$s. Please increase the sort memory budget.
+131 = Result does not follow the schema, %1$s type expected but found %2$s type at '%3$s'
+132 = Extra field in the result, field '%1$s' does not exist at '%2$s' in the schema
+133 = Schema could not be inferred, empty types found in the result
+134 = Schema Limit exceeded, maximum number of heterogeneous schemas allowed : '%1$s'
 
 10000 = The given rule collection %1$s is not an instance of the List class.
 10001 = Cannot compose partition constraint %1$s with %2$s
diff --git a/hyracks-fullstack/hyracks/hyracks-cloud/src/main/java/org/apache/hyracks/cloud/cache/unit/AbstractIndexUnit.java b/hyracks-fullstack/hyracks/hyracks-cloud/src/main/java/org/apache/hyracks/cloud/cache/unit/AbstractIndexUnit.java
index 1532340..54ea23c 100644
--- a/hyracks-fullstack/hyracks/hyracks-cloud/src/main/java/org/apache/hyracks/cloud/cache/unit/AbstractIndexUnit.java
+++ b/hyracks-fullstack/hyracks/hyracks-cloud/src/main/java/org/apache/hyracks/cloud/cache/unit/AbstractIndexUnit.java
@@ -36,7 +36,7 @@
 
     public final void open() {
         lastAccessTime.set(System.nanoTime());
-        openCounter.get();
+        openCounter.incrementAndGet();
     }
 
     public final void close() {
diff --git a/hyracks-fullstack/hyracks/hyracks-cloud/src/main/java/org/apache/hyracks/cloud/filesystem/PhysicalDrive.java b/hyracks-fullstack/hyracks/hyracks-cloud/src/main/java/org/apache/hyracks/cloud/filesystem/PhysicalDrive.java
index 7010e88..28069a9 100644
--- a/hyracks-fullstack/hyracks/hyracks-cloud/src/main/java/org/apache/hyracks/cloud/filesystem/PhysicalDrive.java
+++ b/hyracks-fullstack/hyracks/hyracks-cloud/src/main/java/org/apache/hyracks/cloud/filesystem/PhysicalDrive.java
@@ -53,17 +53,18 @@
 
     @Override
     public boolean computeAndCheckIsPressured() {
-        long usedSpace = getUsedSpace();
+        long usedSpace = getUsedSpace(drivePaths);
         long pressureCapacity = diskSpace.getPressureCapacity();
         boolean isPressured = usedSpace > pressureCapacity;
         this.usedSpace.set(usedSpace);
 
-        if (isPressured) {
-            LOGGER.info("Used space: {}, pressureCapacity: {} (isPressured: {})",
+        if (usedSpace >= diskSpace.getAllocatedCapacity()) {
+            LOGGER.warn(
+                    "Allocated disk space has been exceeded. Used space: {}, pressureCapacity: {} (isPressured: {})",
                     StorageUtil.toHumanReadableSize(usedSpace), StorageUtil.toHumanReadableSize(pressureCapacity),
                     true);
-        } else if (usedSpace >= diskSpace.getAllocatedCapacity()) {
-            LOGGER.warn("Allocated disk space has been exceeded. Used space: {}, pressureCapacity: {}",
+        } else if (isPressured) {
+            LOGGER.info("Used space: {}, pressureCapacity: {} (isPressured: {})",
                     StorageUtil.toHumanReadableSize(usedSpace), StorageUtil.toHumanReadableSize(pressureCapacity),
                     true);
         } else {
@@ -85,10 +86,9 @@
         return usedSpace.get() < diskSpace.getPressureCapacity();
     }
 
-    private long getUsedSpace() {
+    public static long getUsedSpace(List<FileStore> drivePaths) {
         long totalUsedSpace = 0;
-        for (int i = 0; i < drivePaths.size(); i++) {
-            FileStore device = drivePaths.get(i);
+        for (FileStore device : drivePaths) {
             try {
                 totalUsedSpace += getTotalSpace(device) - getUsableSpace(device);
             } catch (HyracksDataException e) {
@@ -121,7 +121,7 @@
         return new DiskSpace(allocatedCapacity, pressureCapacity);
     }
 
-    private static List<FileStore> getDrivePaths(List<IODeviceHandle> deviceHandles) throws HyracksDataException {
+    public static List<FileStore> getDrivePaths(List<IODeviceHandle> deviceHandles) throws HyracksDataException {
         Set<String> distinctDrives = new HashSet<>();
         List<FileStore> fileStores = new ArrayList<>();
         for (IODeviceHandle handle : deviceHandles) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/PartitionProfile.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/PartitionProfile.java
index 9552a26..9a57e30 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/PartitionProfile.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/PartitionProfile.java
@@ -35,6 +35,8 @@
     private long openTime;
 
     private long closeTime;
+    private long totalTime;
+    private long card;
 
     private MultiResolutionEventProfiler mrep;
 
@@ -48,10 +50,13 @@
 
     }
 
-    public PartitionProfile(PartitionId pid, long openTime, long closeTime, MultiResolutionEventProfiler mrep) {
+    public PartitionProfile(PartitionId pid, long openTime, long closeTime, long totalTime, long card,
+            MultiResolutionEventProfiler mrep) {
         this.pid = pid;
         this.openTime = openTime;
         this.closeTime = closeTime;
+        this.totalTime = totalTime;
+        this.card = card;
         this.mrep = mrep;
     }
 
@@ -67,6 +72,10 @@
         return closeTime;
     }
 
+    public long getCardinality() {
+        return card;
+    }
+
     public MultiResolutionEventProfiler getSamples() {
         return mrep;
     }
@@ -75,6 +84,8 @@
     public void writeFields(DataOutput output) throws IOException {
         output.writeLong(closeTime);
         output.writeLong(openTime);
+        output.writeLong(totalTime);
+        output.writeLong(card);
         mrep.writeFields(output);
         pid.writeFields(output);
     }
@@ -83,6 +94,8 @@
     public void readFields(DataInput input) throws IOException {
         closeTime = input.readLong();
         openTime = input.readLong();
+        totalTime = input.readLong();
+        card = input.readLong();
         mrep = MultiResolutionEventProfiler.create(input);
         pid = PartitionId.create(input);
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/TaskProfile.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/TaskProfile.java
index 84376f6..b66be1e 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/TaskProfile.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/TaskProfile.java
@@ -103,6 +103,7 @@
                 ppObj.set("partition-id", pidObj);
                 ppObj.put("open-time", pp.getOpenTime());
                 ppObj.put("close-time", pp.getCloseTime());
+                ppObj.put("cardinality", pp.getCardinality());
                 MultiResolutionEventProfiler samples = pp.getSamples();
                 ppObj.put("offset", samples.getOffset());
                 int resolution = samples.getResolution();
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
index b35111e..960f23b 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
@@ -40,6 +40,7 @@
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Predicate;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hyracks.api.exceptions.ErrorCode;
@@ -67,6 +68,7 @@
      */
     private static final Logger LOGGER = LogManager.getLogger();
     private static final String WORKSPACE_FILE_SUFFIX = ".waf";
+    private static final String STORAGE_ROOT_DIR_NAME = "storage";
     private static final FilenameFilter WORKSPACE_FILES_FILTER = (dir, name) -> name.endsWith(WORKSPACE_FILE_SUFFIX);
     /*
      * Finals
@@ -76,7 +78,7 @@
     private final ExecutorService executor;
     private final BlockingQueue<IoRequest> submittedRequests;
     private final BlockingQueue<IoRequest> freeRequests;
-    private final List<IODeviceHandle> ioDevices;
+    protected final List<IODeviceHandle> ioDevices;
     private final List<IODeviceHandle> workspaces;
     private final IFileDeviceResolver deviceComputer;
     /*
@@ -601,6 +603,36 @@
         ((AbstractBulkOperation) bulkOperation).performOperation();
     }
 
+    @Override
+    public long getSize(Predicate<String> relativePathFilter) {
+        long totalSize = 0;
+
+        // get cached files (read from disk)
+        for (IODeviceHandle deviceHandle : getIODevices()) {
+            FileReference storageRoot = deviceHandle.createFileRef(STORAGE_ROOT_DIR_NAME);
+
+            Set<FileReference> deviceFiles;
+            try {
+                deviceFiles = list(storageRoot, IoUtil.NO_OP_FILTER);
+            } catch (Throwable th) {
+                LOGGER.info("Failed to get local storage files for root {}", storageRoot.getRelativePath(), th);
+                continue;
+            }
+
+            for (FileReference fileReference : deviceFiles) {
+                try {
+                    if (relativePathFilter.test(fileReference.getRelativePath())) {
+                        totalSize += fileReference.getFile().length();
+                    }
+                } catch (Throwable th) {
+                    LOGGER.info("Encountered issue for local storage file {}", fileReference.getRelativePath(), th);
+                }
+            }
+        }
+
+        return totalSize;
+    }
+
     public void setSpaceMaker(IDiskSpaceMaker spaceMaker) {
         this.spaceMaker = spaceMaker;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/profiling/ProfilingPartitionWriterFactory.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/profiling/ProfilingPartitionWriterFactory.java
index e51d2bd..65f3f31 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/profiling/ProfilingPartitionWriterFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/profiling/ProfilingPartitionWriterFactory.java
@@ -24,7 +24,10 @@
 import org.apache.hyracks.api.comm.IPartitionWriterFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
+import org.apache.hyracks.api.dataflow.ProfiledFrameWriter;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
+import org.apache.hyracks.api.job.profiling.OperatorStats;
 import org.apache.hyracks.api.partitions.PartitionId;
 import org.apache.hyracks.control.common.job.profiling.counters.MultiResolutionEventProfiler;
 import org.apache.hyracks.control.common.job.profiling.om.PartitionProfile;
@@ -53,45 +56,42 @@
     public IFrameWriter createFrameWriter(final int receiverIndex) throws HyracksDataException {
         final IFrameWriter writer = new ConnectorSenderProfilingFrameWriter(ctx,
                 delegate.createFrameWriter(receiverIndex), cd.getConnectorId(), senderIndex, receiverIndex);
-        return new IFrameWriter() {
+        return new ProfiledFrameWriter(writer) {
             private long openTime;
-
             private long closeTime;
 
             MultiResolutionEventProfiler mrep = new MultiResolutionEventProfiler(N_SAMPLES);
 
+            IOperatorStats stats = new OperatorStats(cd.getDisplayName(), cd.getConnectorId().toString());
+
             @Override
             public void open() throws HyracksDataException {
+                super.setInputStats(stats);
                 openTime = System.currentTimeMillis();
-                writer.open();
+                super.open();
             }
 
             @Override
             public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
                 mrep.reportEvent();
-                writer.nextFrame(buffer);
-            }
-
-            @Override
-            public void fail() throws HyracksDataException {
-                writer.fail();
+                super.nextFrame(buffer);
             }
 
             @Override
             public void close() throws HyracksDataException {
                 closeTime = System.currentTimeMillis();
-                try {
-                    ((Task) ctx).setPartitionSendProfile(
-                            new PartitionProfile(new PartitionId(ctx.getJobletContext().getJobId(), cd.getConnectorId(),
-                                    senderIndex, receiverIndex), openTime, closeTime, mrep));
-                } finally {
-                    writer.close();
+                long ownTime = getTotalTime();
+                if (stats != null) {
+                    stats.getTimeCounter().set(ownTime);
                 }
-            }
-
-            @Override
-            public void flush() throws HyracksDataException {
-                writer.flush();
+                try {
+                    ((Task) ctx).setPartitionSendProfile(new PartitionProfile(
+                            new PartitionId(ctx.getJobletContext().getJobId(), cd.getConnectorId(), senderIndex,
+                                    receiverIndex),
+                            openTime, closeTime, super.getTotalTime(), stats.getTupleCounter().get(), mrep));
+                } finally {
+                    super.close();
+                }
             }
         };
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
index 6225d4c..410cb009 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
@@ -38,6 +38,7 @@
 import org.apache.hyracks.api.dataflow.IActivity;
 import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
 import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
+import org.apache.hyracks.api.dataflow.ProfiledFrameWriter;
 import org.apache.hyracks.api.dataflow.TaskAttemptId;
 import org.apache.hyracks.api.dataflow.TaskId;
 import org.apache.hyracks.api.dataflow.connectors.IConnectorPolicy;
@@ -182,7 +183,12 @@
                         LOGGER.trace("input: {}: {}", i, conn.getConnectorId());
                         IFrameWriter writer = conn.createPartitioner(task, recordDesc, pwFactory, partition,
                                 partitionCount, td.getOutputPartitionCounts()[i]);
-                        writer = (enforce && !profile) ? EnforceFrameWriter.enforce(writer) : writer;
+                        if (profile) {
+                            //needed to propagate cardinality to the last operator before an exchange
+                            writer = new ProfiledFrameWriter(writer);
+                        } else {
+                            writer = enforce ? EnforceFrameWriter.enforce(writer) : writer;
+                        }
                         operator.setOutputFrameWriter(i, writer, recordDesc);
                     }
                 }
diff --git a/hyracks-fullstack/pom.xml b/hyracks-fullstack/pom.xml
index ec78f7b..0a9ff13 100644
--- a/hyracks-fullstack/pom.xml
+++ b/hyracks-fullstack/pom.xml
@@ -812,7 +812,7 @@
             <dependency>
               <groupId>com.github.javaparser</groupId>
               <artifactId>javaparser-core</artifactId>
-              <version>3.25.5</version>
+              <version>3.26.3</version>
             </dependency>
           </dependencies>
         </plugin>